From c44398212d9b5f3385289443e823cd74a000ac8f Mon Sep 17 00:00:00 2001 From: Karl Pietrzak Date: Fri, 14 Sep 2018 14:26:43 -0400 Subject: [PATCH 001/181] WIP: basic rate() function --- .../AggregateFunctionRate.cpp | 53 ++++++ .../AggregateFunctionRate.h | 158 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 + .../00715_aggregation_rate.reference | 2 + .../0_stateless/00715_aggregation_rate.sql | 16 ++ 5 files changed, 231 insertions(+) create mode 100644 dbms/src/AggregateFunctions/AggregateFunctionRate.cpp create mode 100644 dbms/src/AggregateFunctions/AggregateFunctionRate.h create mode 100644 dbms/tests/queries/0_stateless/00715_aggregation_rate.reference create mode 100644 dbms/tests/queries/0_stateless/00715_aggregation_rate.sql diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRate.cpp b/dbms/src/AggregateFunctions/AggregateFunctionRate.cpp new file mode 100644 index 00000000000..e2d6e0f1aff --- /dev/null +++ b/dbms/src/AggregateFunctions/AggregateFunctionRate.cpp @@ -0,0 +1,53 @@ +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters) +{ + assertNoParameters(name, parameters); + + if (argument_types.size() < 2) + throw Exception("Aggregate function " + name + " requires at least two arguments, with the first being a timestamp", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + + return std::make_shared(argument_types, parameters); +} + +} + +void registerAggregateFunctionRate(AggregateFunctionFactory & factory) +{ + factory.registerFunction("rate", createAggregateFunctionRate, AggregateFunctionFactory::CaseInsensitive); +} + +} diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRate.h b/dbms/src/AggregateFunctions/AggregateFunctionRate.h new file mode 100644 index 00000000000..ededb0f0cb3 --- /dev/null +++ b/dbms/src/AggregateFunctions/AggregateFunctionRate.h @@ -0,0 +1,158 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +struct AggregateFunctionRateData +{ + using TimestampEvent = std::pair; + + bool is_first = false; + + TimestampEvent first_event; + TimestampEvent last_event; + + void add(UInt32 timestamp, Float64 f) + { + if(this->is_first) { + first_event = TimestampEvent{timestamp, f}; + is_first = true; + } else { + last_event = TimestampEvent{timestamp, f}; + } + } + + void merge(const AggregateFunctionRateData & other) + { + // if the arg is earlier than us, replace us with them + if(other.first_event.first < first_event.first) { + first_event = other.first_event; + } + // if the arg is _later_ than us, replace us with them + if(other.last_event.first > last_event.second) { + last_event = other.last_event; + } + + } + void serialize(WriteBuffer & buf) const + { + writeBinary(is_first, buf); + writeBinary(first_event.first, buf); + writeBinary(first_event.second, buf); + + writeBinary(last_event.first, buf); + writeBinary(last_event.second, buf); + } + + void deserialize(ReadBuffer & buf) + { + readBinary(is_first, buf); + + readBinary(first_event.first, buf); + readBinary(first_event.second, buf); + + readBinary(last_event.first, buf); + readBinary(last_event.second, buf); + } +}; + +class AggregateFunctionRate final + : public IAggregateFunctionDataHelper +{ +private: + /* + * implements a basic derivative function + * + * (y2 - y1) / (x2 - x1) + */ + Float64 getRate(const AggregateFunctionRateData & data) const + { + if (data.first_event.first == 0) + return 0; + if(data.last_event.first == 0) + return 0; + // void divide by zero in denominator + if(data.last_event.first == data.first_event.first) + return 0; + + return (data.last_event.second - data.first_event.second) / (data.last_event.first - data.first_event.first); + } + +public: + String getName() const override + { + return "rate"; + } + + AggregateFunctionRate(const DataTypes & arguments, const Array & params) + { + const auto time_arg = arguments.front().get(); + if (!typeid_cast(time_arg) && !typeid_cast(time_arg)) + throw Exception{"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName() + + ", must be DateTime or UInt32"}; + + const auto number_arg = arguments.at(1).get(); + if (!number_arg->isNumber()) + throw Exception{"Illegal type " + number_arg->getName() + " of argument " + toString(1) + " of aggregate function " + + getName() + ", must be a Number", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + } + + + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } + + void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override + { + const auto timestamp = static_cast *>(columns[0])->getData()[row_num]; + const auto value = static_cast *>(columns[1])->getData()[row_num]; + this->data(place).add(timestamp, value); + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + static_cast(to).getData().push_back(getRate(this->data(place))); + } + + const char * getHeaderFilePath() const override + { + return __FILE__; + } +}; + +} diff --git a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp index 3517ad57a73..211ce7e93be 100644 --- a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -15,6 +15,7 @@ void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &); void registerAggregateFunctionsQuantile(AggregateFunctionFactory &); void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory &); void registerAggregateFunctionWindowFunnel(AggregateFunctionFactory &); +void registerAggregateFunctionRate(AggregateFunctionFactory &); void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory &); void registerAggregateFunctionsStatisticsStable(AggregateFunctionFactory &); void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &); @@ -49,6 +50,7 @@ void registerAggregateFunctions() registerAggregateFunctionsQuantile(factory); registerAggregateFunctionsSequenceMatch(factory); registerAggregateFunctionWindowFunnel(factory); + registerAggregateFunctionRate(factory); registerAggregateFunctionsMinMaxAny(factory); registerAggregateFunctionsStatisticsStable(factory); registerAggregateFunctionsStatisticsSimple(factory); diff --git a/dbms/tests/queries/0_stateless/00715_aggregation_rate.reference b/dbms/tests/queries/0_stateless/00715_aggregation_rate.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00715_aggregation_rate.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/0_stateless/00715_aggregation_rate.sql b/dbms/tests/queries/0_stateless/00715_aggregation_rate.sql new file mode 100644 index 00000000000..8cea70a09b1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00715_aggregation_rate.sql @@ -0,0 +1,16 @@ +drop table if exists rate_test; + +create table rate_test (timestamp UInt32, event UInt32) engine=Memory; +insert into rate_test values (0,1000),(1,1001),(2,1002),(3,1003),(4,1004),(5,1005),(6,1006),(7,1007),(8,1008); + +select 1.0 = rate(timestamp, event) from rate_test; + +drop table if exists rate_test2; +create table rate_test2 (uid UInt32 default 1,timestamp DateTime, event UInt32) engine=Memory; +insert into rate_test2(timestamp, event) values ('2018-01-01 01:01:01',1001),('2018-01-01 01:01:02',1002),('2018-01-01 01:01:03',1003),('2018-01-01 01:01:04',1004),('2018-01-01 01:01:05',1005),('2018-01-01 01:01:06',1006),('2018-01-01 01:01:07',1007),('2018-01-01 01:01:08',1008); + + +select 1.0 = rate(timestamp, event ) from rate_test2; + +drop table rate_test; +drop table rate_test2; From dd3f57dd5db87730e7525ad03ace6e66ce47580a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Nov 2018 20:54:53 +0300 Subject: [PATCH 002/181] Fixed some code (incomplete) #3139 --- ...gregateFunctionRate.cpp => AggregateFunctionBoundingRatio.cpp} | 0 .../{AggregateFunctionRate.h => AggregateFunctionBoundingRatio.h} | 0 ..._aggregation_rate.reference => 00715_bounding_ratio.reference} | 0 .../{00715_aggregation_rate.sql => 00715_bounding_ratio.sql} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename dbms/src/AggregateFunctions/{AggregateFunctionRate.cpp => AggregateFunctionBoundingRatio.cpp} (100%) rename dbms/src/AggregateFunctions/{AggregateFunctionRate.h => AggregateFunctionBoundingRatio.h} (100%) rename dbms/tests/queries/0_stateless/{00715_aggregation_rate.reference => 00715_bounding_ratio.reference} (100%) rename dbms/tests/queries/0_stateless/{00715_aggregation_rate.sql => 00715_bounding_ratio.sql} (100%) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRate.cpp b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp similarity index 100% rename from dbms/src/AggregateFunctions/AggregateFunctionRate.cpp rename to dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRate.h b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h similarity index 100% rename from dbms/src/AggregateFunctions/AggregateFunctionRate.h rename to dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h diff --git a/dbms/tests/queries/0_stateless/00715_aggregation_rate.reference b/dbms/tests/queries/0_stateless/00715_bounding_ratio.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00715_aggregation_rate.reference rename to dbms/tests/queries/0_stateless/00715_bounding_ratio.reference diff --git a/dbms/tests/queries/0_stateless/00715_aggregation_rate.sql b/dbms/tests/queries/0_stateless/00715_bounding_ratio.sql similarity index 100% rename from dbms/tests/queries/0_stateless/00715_aggregation_rate.sql rename to dbms/tests/queries/0_stateless/00715_bounding_ratio.sql From accda8bed2fde3699ab9b1a2391b773f949aae63 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 1 Nov 2018 20:55:11 +0300 Subject: [PATCH 003/181] Fixed some code (incomplete) #3139 --- .../AggregateFunctionBoundingRatio.cpp | 29 +-- .../AggregateFunctionBoundingRatio.h | 221 +++++++++--------- .../AggregateFunctionHistogram.cpp | 9 +- 3 files changed, 125 insertions(+), 134 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp index e2d6e0f1aff..88dc5bda29d 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.cpp @@ -1,31 +1,14 @@ -#include -#include - -#include -#include - -#include -#include - -#include -#include -#include - #include -#include -#include +#include #include -#include -#include namespace DB { + namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -34,20 +17,20 @@ namespace AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertNoParameters(name, parameters); + assertBinary(name, argument_types); if (argument_types.size() < 2) - throw Exception("Aggregate function " + name + " requires at least two arguments, with the first being a timestamp", + throw Exception("Aggregate function " + name + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - return std::make_shared(argument_types, parameters); + return std::make_shared(argument_types); } } void registerAggregateFunctionRate(AggregateFunctionFactory & factory) { - factory.registerFunction("rate", createAggregateFunctionRate, AggregateFunctionFactory::CaseInsensitive); + factory.registerFunction("boundingRatio", createAggregateFunctionRate, AggregateFunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index ededb0f0cb3..3cc6d92547b 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -13,146 +13,151 @@ #include #include -#include #include +#include namespace DB { -struct AggregateFunctionRateData +struct AggregateFunctionBoundingRatioData { - using TimestampEvent = std::pair; + using TimestampEvent = std::pair; - bool is_first = false; + bool is_first = false; - TimestampEvent first_event; - TimestampEvent last_event; + TimestampEvent first_event; + TimestampEvent last_event; - void add(UInt32 timestamp, Float64 f) - { - if(this->is_first) { - first_event = TimestampEvent{timestamp, f}; - is_first = true; - } else { - last_event = TimestampEvent{timestamp, f}; - } - } + void add(UInt32 timestamp, Float64 f) + { + if (is_first) + { + first_event = TimestampEvent{timestamp, f}; + is_first = true; + } + else + { + last_event = TimestampEvent{timestamp, f}; + } + } - void merge(const AggregateFunctionRateData & other) - { - // if the arg is earlier than us, replace us with them - if(other.first_event.first < first_event.first) { - first_event = other.first_event; - } - // if the arg is _later_ than us, replace us with them - if(other.last_event.first > last_event.second) { - last_event = other.last_event; - } + void merge(const AggregateFunctionBoundingRatioData & other) + { + // if the arg is earlier than us, replace us with them + if (other.first_event.first < first_event.first) + { + first_event = other.first_event; + } + // if the arg is _later_ than us, replace us with them + if (other.last_event.first > last_event.second) + { + last_event = other.last_event; + } + } - } - void serialize(WriteBuffer & buf) const - { - writeBinary(is_first, buf); - writeBinary(first_event.first, buf); - writeBinary(first_event.second, buf); + void serialize(WriteBuffer & buf) const + { + writeBinary(is_first, buf); + writeBinary(first_event.first, buf); + writeBinary(first_event.second, buf); - writeBinary(last_event.first, buf); - writeBinary(last_event.second, buf); - } + writeBinary(last_event.first, buf); + writeBinary(last_event.second, buf); + } - void deserialize(ReadBuffer & buf) - { - readBinary(is_first, buf); + void deserialize(ReadBuffer & buf) + { + readBinary(is_first, buf); - readBinary(first_event.first, buf); - readBinary(first_event.second, buf); + readBinary(first_event.first, buf); + readBinary(first_event.second, buf); - readBinary(last_event.first, buf); - readBinary(last_event.second, buf); - } + readBinary(last_event.first, buf); + readBinary(last_event.second, buf); + } }; -class AggregateFunctionRate final - : public IAggregateFunctionDataHelper + +class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper { private: - /* - * implements a basic derivative function - * - * (y2 - y1) / (x2 - x1) - */ - Float64 getRate(const AggregateFunctionRateData & data) const - { - if (data.first_event.first == 0) - return 0; - if(data.last_event.first == 0) - return 0; - // void divide by zero in denominator - if(data.last_event.first == data.first_event.first) - return 0; + /* implements a basic derivative function + * + * (y2 - y1) / (x2 - x1) + */ + Float64 getBoundingRatio(const AggregateFunctionBoundingRatioData & data) const + { + if (data.first_event.first == 0) + return 0; + if (data.last_event.first == 0) + return 0; + // void divide by zero in denominator + if (data.last_event.first == data.first_event.first) + return 0; - return (data.last_event.second - data.first_event.second) / (data.last_event.first - data.first_event.first); - } + return (data.last_event.second - data.first_event.second) / (data.last_event.first - data.first_event.first); + } public: - String getName() const override - { - return "rate"; - } + String getName() const override + { + return "boundingRatio"; + } - AggregateFunctionRate(const DataTypes & arguments, const Array & params) - { - const auto time_arg = arguments.front().get(); - if (!typeid_cast(time_arg) && !typeid_cast(time_arg)) - throw Exception{"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName() - + ", must be DateTime or UInt32"}; - - const auto number_arg = arguments.at(1).get(); - if (!number_arg->isNumber()) - throw Exception{"Illegal type " + number_arg->getName() + " of argument " + toString(1) + " of aggregate function " - + getName() + ", must be a Number", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - } + AggregateFunctionBoundingRatio(const DataTypes & arguments) + { - DataTypePtr getReturnType() const override - { - return std::make_shared(); - } + const auto time_arg = arguments.at(0).get(); + if (!typeid_cast(time_arg) && !typeid_cast(time_arg)) + throw Exception {"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName() + + ", must be DateTime or UInt32"}; - void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override - { - const auto timestamp = static_cast *>(columns[0])->getData()[row_num]; - const auto value = static_cast *>(columns[1])->getData()[row_num]; - this->data(place).add(timestamp, value); - } + const auto number_arg = arguments.at(1).get(); + if (!number_arg->isNumber()) + throw Exception {"Illegal type " + number_arg->getName() + " of argument " + toString(1) + " of aggregate function " + getName() + + ", must be a Number", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override - { - this->data(place).merge(this->data(rhs)); - } - void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override - { - this->data(place).serialize(buf); - } + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override - { - this->data(place).deserialize(buf); - } + void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override + { + const auto timestamp = static_cast *>(columns[0])->getData()[row_num]; + const auto value = static_cast *>(columns[1])->getData()[row_num]; + data(place).add(timestamp, value); + } - void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override - { - static_cast(to).getData().push_back(getRate(this->data(place))); - } + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override + { + data(place).merge(data(rhs)); + } - const char * getHeaderFilePath() const override - { - return __FILE__; - } + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override + { + data(place).deserialize(buf); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + static_cast(to).getData().push_back(getBoundingRatio(data(place))); + } + + const char * getHeaderFilePath() const override + { + return __FILE__; + } }; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp index de58d7a36d3..bf46bccd9c7 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp @@ -17,12 +17,13 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; } + namespace { -AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params) +AggregateFunctionPtr createAggregateFunctionHistogram(const std::string &, const DataTypes &, const Array &) { - if (params.size() != 1) +/* if (params.size() != 1) throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (params[0].getType() != Field::Types::UInt64) @@ -43,7 +44,9 @@ AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, if (!res) throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return res; + return res;*/ + + return nullptr; } } From 1b8e5f9b4731817ff90568a3dc318fd77785dcb8 Mon Sep 17 00:00:00 2001 From: Gleb Kanterov Date: Fri, 16 Nov 2018 09:55:16 +0000 Subject: [PATCH 004/181] Optimization to skip unused shards for Distributed engine Hide behind `distributed_optimize_skip_select_on_unused_shards` setting. --- dbms/src/Interpreters/Cluster.cpp | 19 +- dbms/src/Interpreters/Cluster.h | 7 +- dbms/src/Interpreters/Settings.h | 1 + dbms/src/Storages/StorageDistributed.cpp | 9 + .../StorageDistributedShardsOptimizer.cpp | 366 ++++++++++++++++++ .../StorageDistributedShardsOptimizer.h | 25 ++ ...ize_skip_select_on_unused_shards.reference | 4 + ..._optimize_skip_select_on_unused_shards.sql | 17 + dbms/tests/server-test.xml | 14 + 9 files changed, 455 insertions(+), 7 deletions(-) create mode 100644 dbms/src/Storages/StorageDistributedShardsOptimizer.cpp create mode 100644 dbms/src/Storages/StorageDistributedShardsOptimizer.h create mode 100644 dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference create mode 100644 dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 4bf446b107b..f39e2d2fddb 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -397,14 +397,23 @@ void Cluster::initMisc() std::unique_ptr Cluster::getClusterWithSingleShard(size_t index) const { - return std::unique_ptr{ new Cluster(*this, index) }; + return std::unique_ptr{ new Cluster(*this, {index}) }; } -Cluster::Cluster(const Cluster & from, size_t index) - : shards_info{from.shards_info[index]} +std::unique_ptr Cluster::getClusterWithMultipleShards(std::vector indexes) const { - if (!from.addresses_with_failover.empty()) - addresses_with_failover.emplace_back(from.addresses_with_failover[index]); + return std::unique_ptr{ new Cluster(*this, indexes) }; +} + +Cluster::Cluster(const Cluster & from, std::vector indexes) + : shards_info{} +{ + for (size_t index : indexes) { + shards_info.push_back(from.shards_info[index]); + + if (!from.addresses_with_failover.empty()) + addresses_with_failover.emplace_back(from.addresses_with_failover[index]); + } initMisc(); } diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 8bfbc073c61..befd97666e9 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -143,6 +143,9 @@ public: /// Get a subcluster consisting of one shard - index by count (from 0) of the shard of this cluster. std::unique_ptr getClusterWithSingleShard(size_t index) const; + /// Get a subcluster consisting of one or multiple shards - indexes by count (from 0) of the shard of this cluster. + std::unique_ptr getClusterWithMultipleShards(std::vector indexes) const; + private: using SlotToShard = std::vector; SlotToShard slot_to_shard; @@ -153,8 +156,8 @@ public: private: void initMisc(); - /// For getClusterWithSingleShard implementation. - Cluster(const Cluster & from, size_t index); + /// For getClusterWithMultipleShards implementation. + Cluster(const Cluster & from, std::vector indexes); String hash_of_addresses; /// Description of the cluster shards. diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index c1436267ffc..8119315c328 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -89,6 +89,7 @@ struct Settings M(SettingBool, skip_unavailable_shards, false, "Silently skip unavailable shards.") \ \ M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ + M(SettingBool, distributed_optimize_skip_select_on_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.") \ M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.") \ diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 5d3860f449d..a120e922b00 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include #include @@ -269,6 +270,14 @@ BlockInputStreams StorageDistributed::read( : ClusterProxy::SelectStreamFactory( header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables()); + if (settings.distributed_optimize_skip_select_on_unused_shards) { + auto optimizer = StorageDistributedShardsOptimizer(); + auto smaller_cluster = optimizer.skipUnusedShards(cluster, query_info, sharding_key_expr, sharding_key_column_name); + + if (smaller_cluster) + cluster = smaller_cluster; + } + return ClusterProxy::executeQuery( select_stream_factory, cluster, modified_query_ast, context, settings); } diff --git a/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp b/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp new file mode 100644 index 00000000000..10b282e896f --- /dev/null +++ b/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp @@ -0,0 +1,366 @@ +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int TYPE_MISMATCH; +} + +/// Contains a list of columns for conjunction: columns[0] AND columns[1] AND ... +struct Conjunction +{ + ColumnsWithTypeAndName columns; +}; + +/// Contains a list of disjunctions: disjunctions[0] OR disjunctions[1] OR ... +struct Disjunction +{ + std::vector conjunctions; +}; + +using DisjunctionsPtr = std::shared_ptr>; +using Disjunctions = std::vector; + +static constexpr auto and_function_name = "and"; +static constexpr auto equals_function_name = "equals"; +static constexpr auto in_function_name = "in"; +static constexpr auto or_function_name = "or"; +static constexpr auto tuple_function_name = "tuple"; + +void logDebug(std::string message) +{ + LOG_DEBUG(&Logger::get("(StorageDistributedShardsOptimizer)"), message); +} + +/// Returns disjunction equivalent to `disjunctions AND another`. +Disjunctions pairwiseAnd(const Disjunctions & disjunctions, const Disjunctions & another) +{ + Disjunctions new_disjunctions; + + if (disjunctions.empty()) + { + return another; + } + + if (another.empty()) + { + return disjunctions; + } + + for (const auto disjunction : disjunctions) + { + for (const auto another_disjunction : another) + { + std::vector new_conjunctions; + + for (const auto conjunction : disjunction.conjunctions) + { + for (const auto another_conjunction : another_disjunction.conjunctions) + { + ColumnsWithTypeAndName new_columns; + new_columns.insert(std::end(new_columns), conjunction.columns.begin(), conjunction.columns.end()); + new_columns.insert(std::end(new_columns), another_conjunction.columns.begin(), another_conjunction.columns.end()); + + new_conjunctions.push_back(Conjunction{new_columns}); + } + } + + new_disjunctions.push_back(Disjunction{new_conjunctions}); + } + } + + return new_disjunctions; +} + +/// Given `ident = literal` expr, returns disjunctions relevant for constant folding in sharding_key_expr. +DisjunctionsPtr analyzeEquals(const ASTIdentifier * ident, const ASTLiteral * literal, ExpressionActionsPtr sharding_key_expr) +{ + for (const auto name_and_type : sharding_key_expr->getRequiredColumnsWithTypes()) + { + const auto type = name_and_type.type; + const auto name = name_and_type.name; + + if (name == ident->name) + { + ColumnWithTypeAndName column; + + column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); + column.type = type; + column.name = name; + + const auto columns = ColumnsWithTypeAndName{column}; + const auto conjunction = Conjunction{columns}; + const auto disjunction = Disjunction{{conjunction}}; + const Disjunctions disjunctions = {disjunction}; + + return std::make_shared(disjunctions); + } + } + + const Disjunctions disjunctions = {}; + return std::make_shared(disjunctions); +} + +/// Given `ident IN (..literals)` expr, returns disjunctions relevant for constant folding in sharding_key_expr. +DisjunctionsPtr analyzeIn( + const ASTIdentifier * ident, const std::vector literals, ExpressionActionsPtr sharding_key_expr) +{ + Disjunctions disjunctions; + + for (const auto literal : literals) + { + const auto inner_disjunctions = analyzeEquals(ident, literal, sharding_key_expr); + + if (!inner_disjunctions) + return nullptr; + + disjunctions.insert(std::end(disjunctions), inner_disjunctions->begin(), inner_disjunctions->end()); + } + + return std::make_shared(disjunctions); +} + +/// Given WHERE condition, returns disjunctions relevant for constant folding in sharding_key_expr. +DisjunctionsPtr analyzeQuery(const ASTFunction * function, ExpressionActionsPtr sharding_key_expr) +{ + if (function->name == equals_function_name) + { + auto left_arg = function->arguments->children.front().get(); + auto right_arg = function->arguments->children.back().get(); + + // try to ensure left_arg points to ASTIdentifier + if (!typeid_cast(left_arg) && typeid_cast(right_arg)) + std::swap(left_arg, right_arg); + + const auto ident = typeid_cast(left_arg); + const auto literal = typeid_cast(right_arg); + + if (!ident || !literal) + { + logDebug("didn't match pattern ident = "); + return nullptr; + } + + return analyzeEquals(ident, literal, sharding_key_expr); + } + else if (function->name == in_function_name) + { + const auto left_arg = function->arguments->children.front().get(); + const auto right_arg = function->arguments->children.back().get(); + + const auto ident = typeid_cast(left_arg); + const auto inner_function = typeid_cast(right_arg); + + if (!ident || !inner_function || inner_function->name != tuple_function_name) + { + logDebug("didn't match pattern ident IN tuple(...)"); + return nullptr; + } + + std::vector literals; + const auto expr_list = typeid_cast(inner_function->children.front().get()); + + if (!expr_list) + { + logDebug("expected ExpressionList in tuple, got: " + inner_function->getID()); + return nullptr; + } + + for (const auto child : expr_list->children) + { + if (const auto child_literal = typeid_cast(child.get())) + { + literals.push_back(child_literal); + } + else + { + logDebug("non-literal in IN expression, got: " + child->getID()); + return nullptr; + } + } + + return analyzeIn(ident, literals, sharding_key_expr); + } + else if (function->name == or_function_name) + { + const auto expr_list = typeid_cast(function->children.front().get()); + + if (!expr_list) + { + logDebug("expected ExpressionList in IN, got: " + function->getID()); + return nullptr; + } + + Disjunctions disjunctions; + + for (const auto child : expr_list->children) + { + // we can't ignore expr we can't analyze because it can widden the set of shards + if (const auto child_function = typeid_cast(child.get())) + { + const auto child_disjunctions = analyzeQuery(child_function, sharding_key_expr); + + if (!child_disjunctions) + return nullptr; + + disjunctions.insert(std::end(disjunctions), child_disjunctions->begin(), child_disjunctions->end()); + } + else + { + logDebug("non-function expression in OR, got: " + child->getID()); + return nullptr; + } + } + + return std::make_shared(disjunctions); + } + else if (function->name == and_function_name) + { + const auto expr_list = typeid_cast(function->children.front().get()); + + if (!expr_list) + { + logDebug("expected ExpressionList in AND, got: " + function->getID()); + return nullptr; + } + + Disjunctions disjunctions; + + for (const auto child : expr_list->children) + { + // we can skip everything we can't analyze because it can only narrow the set of shards + if (const auto child_function = typeid_cast(child.get())) + { + const auto child_disjunctions = analyzeQuery(child_function, sharding_key_expr); + + if (!child_disjunctions) + continue; + + disjunctions = pairwiseAnd(disjunctions, *child_disjunctions); + } + } + + return std::make_shared(disjunctions); + } + else + { + logDebug("unsupported function: " + function->name); + return nullptr; + } +} + +/// the same as DistributedBlockOutputStream::createSelector, should it be static? +IColumn::Selector createSelector(const ClusterPtr cluster, const ColumnWithTypeAndName & result) +{ + const auto & slot_to_shard = cluster->getSlotToShard(); + +#define CREATE_FOR_TYPE(TYPE) \ + if (typeid_cast(result.type.get())) \ + return createBlockSelector(*result.column, slot_to_shard); + + CREATE_FOR_TYPE(UInt8) + CREATE_FOR_TYPE(UInt16) + CREATE_FOR_TYPE(UInt32) + CREATE_FOR_TYPE(UInt64) + CREATE_FOR_TYPE(Int8) + CREATE_FOR_TYPE(Int16) + CREATE_FOR_TYPE(Int32) + CREATE_FOR_TYPE(Int64) + +#undef CREATE_FOR_TYPE + + throw Exception{"Sharding key expression does not evaluate to an integer type", ErrorCodes::TYPE_MISMATCH}; +} + +/// Returns true if block has all columns required by sharding_key_expr. +bool hasRequiredColumns(const Block & block, ExpressionActionsPtr sharding_key_expr) +{ + for (const auto name : sharding_key_expr->getRequiredColumns()) + { + bool hasColumn = false; + for (const auto column_name : block.getNames()) + { + if (column_name == name) + { + hasColumn = true; + break; + } + } + + if (!hasColumn) + return false; + } + + return true; +} + +StorageDistributedShardsOptimizer::StorageDistributedShardsOptimizer() {} + +/** Returns a new cluster with fewer shards if constant folding for sharding_key_expr is possible + * using constraints from WHERE condition, otherwise, returns nullptr. */ +ClusterPtr StorageDistributedShardsOptimizer::skipUnusedShards( + ClusterPtr cluster, const SelectQueryInfo & query_info, ExpressionActionsPtr sharding_key_expr, std::string sharding_key_column_name) +{ + const auto & select = typeid_cast(*query_info.query); + + if (!select.where_expression) + return nullptr; + + const auto function = typeid_cast(select.where_expression.get()); + + if (!function) + return nullptr; + + const auto disjunctions = analyzeQuery(function, sharding_key_expr); + + // didn't get definite answer from analysis, about optimization + if (!disjunctions) + return nullptr; + + std::set shards; + + for (const auto disjunction : *disjunctions) + { + for (const auto conjunction : disjunction.conjunctions) + { + Block block(conjunction.columns); + + // check if sharding_key_expr requires column that we don't know anything about + // if so, we don't have enough information to optimize + if (!hasRequiredColumns(block, sharding_key_expr)) + return nullptr; + + sharding_key_expr->execute(block); + + if (!block || block.rows() != 1 || !block.has(sharding_key_column_name)) + throw Exception("Logical error: sharding_key_expr should evaluate as 1 row"); + + const auto result = block.getByName(sharding_key_column_name); + const auto selector = createSelector(cluster, result); + + shards.insert(selector.begin(), selector.end()); + } + } + + return cluster->getClusterWithMultipleShards({shards.begin(), shards.end()}); +} +} diff --git a/dbms/src/Storages/StorageDistributedShardsOptimizer.h b/dbms/src/Storages/StorageDistributedShardsOptimizer.h new file mode 100644 index 00000000000..f6dee543b71 --- /dev/null +++ b/dbms/src/Storages/StorageDistributedShardsOptimizer.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include +#include + +#include + +namespace Poco +{ +class Logger; +} + +namespace DB +{ +class StorageDistributedShardsOptimizer : private boost::noncopyable +{ +public: + StorageDistributedShardsOptimizer(); + ClusterPtr skipUnusedShards(ClusterPtr cluster, + const SelectQueryInfo & query_info, + ExpressionActionsPtr sharding_key_expr, + std::string sharding_key_column_name); +}; +} diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference new file mode 100644 index 00000000000..44e0be8e356 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference @@ -0,0 +1,4 @@ +0 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql new file mode 100644 index 00000000000..13f88ead4b5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql @@ -0,0 +1,17 @@ +SET distributed_optimize_skip_select_on_unused_shards = 1; + +DROP TABLE IF EXISTS test.mergetree; +DROP TABLE IF EXISTS test.distributed; + +CREATE TABLE test.mergetree (a Int64, b Int64) ENGINE = MergeTree ORDER BY (a, b); +CREATE TABLE test.distributed AS test.mergetree ENGINE = Distributed(test_unavailable_shard, test, mergetree, jumpConsistentHash(a+b, 2)); + +INSERT INTO test.mergetree VALUES (0, 0); +INSERT INTO test.mergetree VALUES (1, 0); +INSERT INTO test.mergetree VALUES (0, 1); +INSERT INTO test.mergetree VALUES (1, 1); + +/* without setting, quering of the second shard will fail because it isn't available */ + +SELECT jumpConsistentHash(a+b, 2) FROM test.distributed +WHERE (a+b > 0 AND a = 0 AND b = 0) OR (a IN (0, 1) AND b IN (0, 1)) OR ((a = 1 OR a = 1) AND b = 2); diff --git a/dbms/tests/server-test.xml b/dbms/tests/server-test.xml index cb9d63aa94c..5d0d1281c6b 100644 --- a/dbms/tests/server-test.xml +++ b/dbms/tests/server-test.xml @@ -53,6 +53,20 @@ Europe/Moscow + + + + localhost + 59000 + + + + + 127.0.0.255 + 59000 + + + From 866917ad3249454ea2ea44d7a250a4e007a26c90 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Nov 2018 07:02:19 +0300 Subject: [PATCH 005/181] Update Cluster.cpp --- dbms/src/Interpreters/Cluster.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index f39e2d2fddb..e76ec26f385 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -400,15 +400,16 @@ std::unique_ptr Cluster::getClusterWithSingleShard(size_t index) const return std::unique_ptr{ new Cluster(*this, {index}) }; } -std::unique_ptr Cluster::getClusterWithMultipleShards(std::vector indexes) const +std::unique_ptr Cluster::getClusterWithMultipleShards(std::vector indices) const { - return std::unique_ptr{ new Cluster(*this, indexes) }; + return std::unique_ptr{ new Cluster(*this, indices) }; } -Cluster::Cluster(const Cluster & from, std::vector indexes) +Cluster::Cluster(const Cluster & from, std::vector indices) : shards_info{} { - for (size_t index : indexes) { + for (size_t index : indices) + { shards_info.push_back(from.shards_info[index]); if (!from.addresses_with_failover.empty()) From 5cf9beabe36531e50bc41a360986f0f28984ec90 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Nov 2018 07:02:56 +0300 Subject: [PATCH 006/181] Update StorageDistributed.cpp --- dbms/src/Storages/StorageDistributed.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index a120e922b00..f0cc2a7292e 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -270,7 +270,8 @@ BlockInputStreams StorageDistributed::read( : ClusterProxy::SelectStreamFactory( header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables()); - if (settings.distributed_optimize_skip_select_on_unused_shards) { + if (settings.distributed_optimize_skip_select_on_unused_shards) + { auto optimizer = StorageDistributedShardsOptimizer(); auto smaller_cluster = optimizer.skipUnusedShards(cluster, query_info, sharding_key_expr, sharding_key_column_name); From 5ee36769db2f31854afed821ef9f982f4b766563 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Nov 2018 07:04:05 +0300 Subject: [PATCH 007/181] Update Cluster.cpp --- dbms/src/Interpreters/Cluster.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index e76ec26f385..9b090a2a9e6 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -400,12 +400,12 @@ std::unique_ptr Cluster::getClusterWithSingleShard(size_t index) const return std::unique_ptr{ new Cluster(*this, {index}) }; } -std::unique_ptr Cluster::getClusterWithMultipleShards(std::vector indices) const +std::unique_ptr Cluster::getClusterWithMultipleShards(const std::vector & indices) const { return std::unique_ptr{ new Cluster(*this, indices) }; } -Cluster::Cluster(const Cluster & from, std::vector indices) +Cluster::Cluster(const Cluster & from, const std::vector & indices) : shards_info{} { for (size_t index : indices) From 96fdc2a39023585de99db69a801fd84993958e9e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Nov 2018 07:04:24 +0300 Subject: [PATCH 008/181] Update Cluster.h --- dbms/src/Interpreters/Cluster.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index befd97666e9..e6e462adbc5 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -144,7 +144,7 @@ public: std::unique_ptr getClusterWithSingleShard(size_t index) const; /// Get a subcluster consisting of one or multiple shards - indexes by count (from 0) of the shard of this cluster. - std::unique_ptr getClusterWithMultipleShards(std::vector indexes) const; + std::unique_ptr getClusterWithMultipleShards(const std::vector & indices) const; private: using SlotToShard = std::vector; From af8db3e9ec7d5c4b83ccd1215ac188d234aa7982 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Nov 2018 07:04:53 +0300 Subject: [PATCH 009/181] Update Cluster.h --- dbms/src/Interpreters/Cluster.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index e6e462adbc5..f2a8f70d271 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -157,7 +157,7 @@ private: void initMisc(); /// For getClusterWithMultipleShards implementation. - Cluster(const Cluster & from, std::vector indexes); + Cluster(const Cluster & from, const std::vector & indices); String hash_of_addresses; /// Description of the cluster shards. From 0bc9776e2a3d2f18f11df258f263eabc606b4770 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 21 Nov 2018 07:06:40 +0300 Subject: [PATCH 010/181] Update Cluster.cpp --- dbms/src/Interpreters/Cluster.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 9b090a2a9e6..dbfad8c4255 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -410,10 +410,10 @@ Cluster::Cluster(const Cluster & from, const std::vector & indices) { for (size_t index : indices) { - shards_info.push_back(from.shards_info[index]); + shards_info.emplace_back(from.shards_info.at(index)); if (!from.addresses_with_failover.empty()) - addresses_with_failover.emplace_back(from.addresses_with_failover[index]); + addresses_with_failover.emplace_back(from.addresses_with_failover.at(index)); } initMisc(); From b3e694e3ebc5210af184b1212a6740e9124f66d6 Mon Sep 17 00:00:00 2001 From: Persiyanov Dmitriy Andreevich Date: Wed, 21 Nov 2018 13:10:22 +0300 Subject: [PATCH 011/181] ugly version --- dbms/src/Functions/FunctionsArithmetic.h | 6 +-- dbms/src/Functions/FunctionsRound.cpp | 1 + dbms/src/Functions/FunctionsRound.h | 58 ++++++++++++++++++++++++ 3 files changed, 62 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index ab0b5a1ec00..5e3731bd0f9 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -128,7 +128,7 @@ struct PlusImpl return static_cast(a) + b; } - /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise. + /// Apply operation and check overflow. It's used for Decimal operations. @returns true if overflowed, false otherwise. template static inline bool apply(A a, B b, Result & c) { @@ -158,7 +158,7 @@ struct MultiplyImpl return static_cast(a) * b; } - /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise. + /// Apply operation and check overflow. It's used for Decimal operations. @returns true if overflowed, false otherwise. template static inline bool apply(A a, B b, Result & c) { @@ -187,7 +187,7 @@ struct MinusImpl return static_cast(a) - b; } - /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise. + /// Apply operation and check overflow. It's used for Decimal operations. @returns true if overflowed, false otherwise. template static inline bool apply(A a, B b, Result & c) { diff --git a/dbms/src/Functions/FunctionsRound.cpp b/dbms/src/Functions/FunctionsRound.cpp index 9cb9e1001ae..ec7868f7aff 100644 --- a/dbms/src/Functions/FunctionsRound.cpp +++ b/dbms/src/Functions/FunctionsRound.cpp @@ -9,6 +9,7 @@ void registerFunctionsRound(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction("round", FunctionFactory::CaseInsensitive); factory.registerFunction("floor", FunctionFactory::CaseInsensitive); diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 2c941d8255b..6f4a7473361 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; } @@ -665,6 +666,63 @@ public: }; +class FunctionRoundDown : public IFunction { +public: + static constexpr auto name = "roundDown"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + +public: + String getName() const override + { + return name; + } + + bool isVariadic() const override { return false; } + size_t getNumberOfArguments() const override { return 2; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { + const ColumnConst * array = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); + + if (!array) + throw Exception{"Second argument of function " + getName() + " must be constant array.", ErrorCodes::ILLEGAL_COLUMN}; + + const Array & boundaries = array->getValue(); + + const auto src = checkAndGetColumn>(block.getByPosition(arguments.front()).column.get())->getdata(); + auto column_result = block.getByPosition(result).type->createColumn(); + auto dst = checkAndGetColumn>(column_result.get())->getData(); + + size_t size = src.size(); + size_t boundaries_size = boundaries.size(); + dst.resize(size); + for (size_t i = 0; i < size; ++i) + { + if (src[i] < boundaries[0].get()) + { + dst[i] = boundaries[0].get(); + } + else if (src[i] >= boundaries.back().get()) { + dst[i] = boundaries.back().get(); + } + else + { + for (size_t j = 1; j < boundaries_size; ++j) { + if (src[i] < boundaries[i].get()) + { + dst[i] = boundaries[i - 1].get(); + break; + } + } + } + } + + block.getByPosition(result).column = std::move(column_result); + } + +}; + + struct NameRoundToExp2 { static constexpr auto name = "roundToExp2"; }; struct NameRoundDuration { static constexpr auto name = "roundDuration"; }; struct NameRoundAge { static constexpr auto name = "roundAge"; }; From 75919bd91e5990261b388f1e843e1e2e59082a4e Mon Sep 17 00:00:00 2001 From: Gleb Kanterov Date: Wed, 28 Nov 2018 19:44:50 +0100 Subject: [PATCH 012/181] Update server/config.xml for tests - add test_unavailable_shard cluster --- dbms/programs/server/config.xml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 051f6f7fb2b..6218f69f184 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -173,6 +173,20 @@ + + + + localhost + 9000 + + + + + 127.0.0.255 + 9000 + + + From 614af51fb7ee9bf170f12b370bdbdd52037071d4 Mon Sep 17 00:00:00 2001 From: Persiyanov Dmitriy Andreevich Date: Thu, 29 Nov 2018 03:32:52 +0300 Subject: [PATCH 013/181] test --- dbms/src/DataTypes/IDataType.h | 2 +- dbms/src/Functions/FunctionsRound.h | 40 ++++++++++++++----- .../00161_rounding_functions.reference | 11 +++++ .../0_stateless/00161_rounding_functions.sql | 3 ++ 4 files changed, 46 insertions(+), 10 deletions(-) diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 727d80540ce..eeac69ec545 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -26,7 +26,7 @@ using DataTypes = std::vector; /** Properties of data type. * Contains methods for serialization/deserialization. * Implementations of this interface represent a data type (example: UInt8) - * or parapetric family of data types (example: Array(...)). + * or parametric family of data types (example: Array(...)). * * DataType is totally immutable object. You can always share them. */ diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 6f4a7473361..bc684402862 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -10,6 +11,8 @@ #include #include +#include + #if __SSE4_1__ #include #endif @@ -669,7 +672,7 @@ public: class FunctionRoundDown : public IFunction { public: static constexpr auto name = "roundDown"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context &) { return std::make_shared(); } public: String getName() const override @@ -681,17 +684,38 @@ public: size_t getNumberOfArguments() const override { return 2; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - const ColumnConst * array = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + return arguments[0]; + } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) override { + std::cerr << "arguments.size() = " << arguments.size() << std::endl; + const ColumnConst * array = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); + std::cerr << "got array" << std::endl; if (!array) throw Exception{"Second argument of function " + getName() + " must be constant array.", ErrorCodes::ILLEGAL_COLUMN}; - + std::cerr << "checked array" << std::endl; const Array & boundaries = array->getValue(); - - const auto src = checkAndGetColumn>(block.getByPosition(arguments.front()).column.get())->getdata(); + std::cerr << "got boundaries" << std::endl; + const auto in = block.getByPosition(arguments.front()).column.get(); + std::cerr << "got in" << std::endl; auto column_result = block.getByPosition(result).type->createColumn(); - auto dst = checkAndGetColumn>(column_result.get())->getData(); + std::cerr << "got column result" << std::endl; + auto out = column_result.get(); + std::cerr << "got out" << std::endl; + + executeNum(in, out, boundaries); + std::cerr << "executeNum done" << std::endl; + + block.getByPosition(result).column = std::move(column_result); + std::cerr << "assigned result" << std::endl; + } + +private: + template + void executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) { + const auto & src = checkAndGetColumn>(in_untyped)->getData(); + auto & dst = typeid_cast *>(out_untyped)->getData(); size_t size = src.size(); size_t boundaries_size = boundaries.size(); @@ -717,9 +741,7 @@ public: } } - block.getByPosition(result).column = std::move(column_result); } - }; diff --git a/dbms/tests/queries/0_stateless/00161_rounding_functions.reference b/dbms/tests/queries/0_stateless/00161_rounding_functions.reference index 9fa1965657e..fbfcaa2b754 100644 --- a/dbms/tests/queries/0_stateless/00161_rounding_functions.reference +++ b/dbms/tests/queries/0_stateless/00161_rounding_functions.reference @@ -682,3 +682,14 @@ 12345.6789 12340 12300 12000 10000 0 12345.6 12345.67 12345.678 12345.6789 12345.6789 64 64 2 0 0 0.5 0 -0.5 -0.5 -0.125 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 5 +7 5 +8 5 +9 5 +10 5 diff --git a/dbms/tests/queries/0_stateless/00161_rounding_functions.sql b/dbms/tests/queries/0_stateless/00161_rounding_functions.sql index 4930d9d0ace..a8a67d5ed20 100644 --- a/dbms/tests/queries/0_stateless/00161_rounding_functions.sql +++ b/dbms/tests/queries/0_stateless/00161_rounding_functions.sql @@ -43,3 +43,6 @@ SELECT 12345.6789 AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), SELECT roundToExp2(100), roundToExp2(64), roundToExp2(3), roundToExp2(0), roundToExp2(-1); SELECT roundToExp2(0.9), roundToExp2(0), roundToExp2(-0.5), roundToExp2(-0.6), roundToExp2(-0.2); + + +SELECT toInt32(number) as x, roundDown(x, [0, 1, 2, 3, 4, 5]) FROM system.numbers LIMIT 10; \ No newline at end of file From 5d3034c56061338df9964d0040491af24fd8bf62 Mon Sep 17 00:00:00 2001 From: Persiyanov Dmitriy Andreevich Date: Fri, 30 Nov 2018 14:15:58 +0300 Subject: [PATCH 014/181] almost works w/ different types --- dbms/src/Functions/FunctionsRound.h | 123 ++++++++++++++++++++++------ 1 file changed, 96 insertions(+), 27 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index bc684402862..0885fbfa760 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include @@ -685,62 +687,129 @@ public: ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - return arguments[0]; + const DataTypePtr & type_x = arguments[0]; + + if (!isNumber(type_x)) + throw Exception{"Unsupported type " + type_x->getName() + + " of first argument of function " + getName() + + ", must be numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + const DataTypeArray * type_arr = checkAndGetDataType(arguments[1].get()); + + if (!type_arr) + throw Exception{"Second argument of function " + getName() + + ", must be array of boundaries to round to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + const auto type_arr_nested = type_arr->getNestedType(); + + if (!isNumber(type_arr_nested)) + { + throw Exception{"Elements of array of second argument of function " + getName() + + " must be numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + return getLeastSupertype({type_x, type_arr_nested}); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) override { - std::cerr << "arguments.size() = " << arguments.size() << std::endl; + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { const ColumnConst * array = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); - std::cerr << "got array" << std::endl; if (!array) + { throw Exception{"Second argument of function " + getName() + " must be constant array.", ErrorCodes::ILLEGAL_COLUMN}; - std::cerr << "checked array" << std::endl; - const Array & boundaries = array->getValue(); - std::cerr << "got boundaries" << std::endl; - const auto in = block.getByPosition(arguments.front()).column.get(); - std::cerr << "got in" << std::endl; - auto column_result = block.getByPosition(result).type->createColumn(); - std::cerr << "got column result" << std::endl; - auto out = column_result.get(); - std::cerr << "got out" << std::endl; + } - executeNum(in, out, boundaries); - std::cerr << "executeNum done" << std::endl; + const Array & boundaries = array->getValue(); + const auto in = block.getByPosition(arguments[0]).column.get(); + + if (in->isColumnConst()) + { + executeConst(block, arguments, result, input_rows_count); + return; + } + + auto column_result = block.getByPosition(result).type->createColumn(); + auto out = column_result.get(); + + if (!executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries) + && !executeNum(in, out, boundaries)) + { + throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } block.getByPosition(result).column = std::move(column_result); - std::cerr << "assigned result" << std::endl; } private: - template - void executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) { - const auto & src = checkAndGetColumn>(in_untyped)->getData(); - auto & dst = typeid_cast *>(out_untyped)->getData(); + void executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { + /// Materialize the input column and compute the function as usual. + Block tmp_block; + ColumnNumbers tmp_arguments; + + tmp_block.insert(block.getByPosition(arguments[0])); + tmp_block.getByPosition(0).column = tmp_block.getByPosition(0).column->cloneResized(input_rows_count)->convertToFullColumnIfConst(); + tmp_arguments.push_back(0); + + for (size_t i = 1; i < arguments.size(); ++i) + { + tmp_block.insert(block.getByPosition(arguments[i])); + tmp_arguments.push_back(i); + } + + tmp_block.insert(block.getByPosition(result)); + size_t tmp_result = arguments.size(); + + execute(tmp_block, tmp_arguments, tmp_result, input_rows_count); + + block.getByPosition(result).column = tmp_block.getByPosition(tmp_result).column; + } + + template + bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) { + const auto in = checkAndGetColumn>(in_untyped); + auto out = typeid_cast *>(out_untyped); + if (!in || !out) + { + return false; + } + + executeImplNumToNum(in->getData(), out->getData(), boundaries); + return true; + } + + template + void executeImplNumToNum(const PaddedPODArray & src, PaddedPODArray & dst, const Array & boundaries) { size_t size = src.size(); size_t boundaries_size = boundaries.size(); dst.resize(size); for (size_t i = 0; i < size; ++i) { - if (src[i] < boundaries[0].get()) + if (src[i] <= boundaries[0].get()) { - dst[i] = boundaries[0].get(); + dst[i] = boundaries[0].get(); } - else if (src[i] >= boundaries.back().get()) { - dst[i] = boundaries.back().get(); + else if (src[i] >= boundaries.back().get()) + { + dst[i] = boundaries.back().get(); } else { for (size_t j = 1; j < boundaries_size; ++j) { - if (src[i] < boundaries[i].get()) + if (src[i] < boundaries[i].get()) { - dst[i] = boundaries[i - 1].get(); + dst[i] = boundaries[i - 1].get(); break; } } } } - } }; From 4a48414a5c829a5d5728f6a999c560e945ae1294 Mon Sep 17 00:00:00 2001 From: Persiyanov Dmitriy Andreevich Date: Fri, 30 Nov 2018 18:57:06 +0300 Subject: [PATCH 015/181] working with numeric types --- dbms/src/Functions/FunctionsRound.h | 55 ++++++++++++++++++----------- 1 file changed, 35 insertions(+), 20 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 0885fbfa760..417df854741 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -6,14 +6,15 @@ #include #include #include +#include +#include #include #include #include #include #include - -#include +#include #if __SSE4_1__ #include @@ -674,7 +675,8 @@ public: class FunctionRoundDown : public IFunction { public: static constexpr auto name = "roundDown"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionRoundDown(const Context & context) : context(context) {} public: String getName() const override @@ -717,18 +719,30 @@ public: throw Exception{"Second argument of function " + getName() + " must be constant array.", ErrorCodes::ILLEGAL_COLUMN}; } - const Array & boundaries = array->getValue(); - const auto in = block.getByPosition(arguments[0]).column.get(); + auto in_column = block.getByPosition(arguments[0]).column; + const auto & in_type = block.getByPosition(arguments[0]).type; - if (in->isColumnConst()) + if (in_column->isColumnConst()) { executeConst(block, arguments, result, input_rows_count); return; } - auto column_result = block.getByPosition(result).type->createColumn(); + const auto & return_type = block.getByPosition(result).type; + auto column_result = return_type->createColumn(); auto out = column_result.get(); + if (!in_type->equals(*return_type)) + { + in_column = castColumn(block.getByPosition(arguments[0]), return_type, context); + } + + const auto in = in_column.get(); + auto boundaries = array->getValue(); + for (size_t i = 0; i < boundaries.size(); ++i) { + boundaries[i] = convertFieldToType(boundaries[i], *return_type); + } + if (!executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) @@ -779,38 +793,39 @@ private: { return false; } - executeImplNumToNum(in->getData(), out->getData(), boundaries); return true; } template void executeImplNumToNum(const PaddedPODArray & src, PaddedPODArray & dst, const Array & boundaries) { + PaddedPODArray bvalues(boundaries.size()); + for (size_t i = 0; i < boundaries.size(); ++i) { + bvalues[i] = boundaries[i].get(); + } + size_t size = src.size(); - size_t boundaries_size = boundaries.size(); dst.resize(size); for (size_t i = 0; i < size; ++i) { - if (src[i] <= boundaries[0].get()) + auto it = std::upper_bound(bvalues.begin(), bvalues.end(), src[i]); + if (it == bvalues.end()) { - dst[i] = boundaries[0].get(); + dst[i] = bvalues.back(); } - else if (src[i] >= boundaries.back().get()) + else if (it == bvalues.begin()) { - dst[i] = boundaries.back().get(); + dst[i] = bvalues.front(); } else { - for (size_t j = 1; j < boundaries_size; ++j) { - if (src[i] < boundaries[i].get()) - { - dst[i] = boundaries[i - 1].get(); - break; - } - } + dst[i] = *(it - 1); } } } + +private: + const Context & context; }; From d982d39b1d7b183d234a3064b7e731668ffd9aab Mon Sep 17 00:00:00 2001 From: Persiyanov Dmitriy Andreevich Date: Fri, 30 Nov 2018 19:13:40 +0300 Subject: [PATCH 016/181] remove NameRoundToExp2 & co --- dbms/src/Functions/FunctionsRound.h | 4 ---- dbms/tests/queries/0_stateless/00161_rounding_functions.sql | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 7fd8c09be79..df21116db46 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -719,10 +719,6 @@ private: }; -struct NameRoundToExp2 { static constexpr auto name = "roundToExp2"; }; -struct NameRoundDuration { static constexpr auto name = "roundDuration"; }; -struct NameRoundAge { static constexpr auto name = "roundAge"; }; - struct NameRound { static constexpr auto name = "round"; }; struct NameCeil { static constexpr auto name = "ceil"; }; struct NameFloor { static constexpr auto name = "floor"; }; diff --git a/dbms/tests/queries/0_stateless/00161_rounding_functions.sql b/dbms/tests/queries/0_stateless/00161_rounding_functions.sql index a8a67d5ed20..0b8c164bb4f 100644 --- a/dbms/tests/queries/0_stateless/00161_rounding_functions.sql +++ b/dbms/tests/queries/0_stateless/00161_rounding_functions.sql @@ -45,4 +45,4 @@ SELECT roundToExp2(100), roundToExp2(64), roundToExp2(3), roundToExp2(0), roundT SELECT roundToExp2(0.9), roundToExp2(0), roundToExp2(-0.5), roundToExp2(-0.6), roundToExp2(-0.2); -SELECT toInt32(number) as x, roundDown(x, [0, 1, 2, 3, 4, 5]) FROM system.numbers LIMIT 10; \ No newline at end of file +SELECT toInt32(number) as x, roundDown(x, [0, 1, 2, 3, 4, 5]) FROM system.numbers LIMIT 10; From 4ec2a2792ca99781d297fb92020ff5ed022b324d Mon Sep 17 00:00:00 2001 From: Persiyanov Dmitriy Andreevich Date: Mon, 3 Dec 2018 14:28:22 +0300 Subject: [PATCH 017/181] style guide --- dbms/src/Functions/FunctionsRound.h | 29 +++++++++++++++++------------ 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index df21116db46..bb29b92471a 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -562,23 +562,22 @@ public: }; -class FunctionRoundDown : public IFunction { +class FunctionRoundDown : public IFunction +{ public: static constexpr auto name = "roundDown"; static FunctionPtr create(const Context & context) { return std::make_shared(context); } FunctionRoundDown(const Context & context) : context(context) {} public: - String getName() const override - { - return name; - } + String getName() const override { return name; } bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 2; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { const DataTypePtr & type_x = arguments[0]; if (!isNumber(type_x)) @@ -602,7 +601,8 @@ public: return getLeastSupertype({type_x, type_arr_nested}); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { const ColumnConst * array = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); if (!array) { @@ -629,7 +629,8 @@ public: const auto in = in_column.get(); auto boundaries = array->getValue(); - for (size_t i = 0; i < boundaries.size(); ++i) { + for (size_t i = 0; i < boundaries.size(); ++i) + { boundaries[i] = convertFieldToType(boundaries[i], *return_type); } @@ -651,7 +652,8 @@ public: } private: - void executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { + void executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) + { /// Materialize the input column and compute the function as usual. Block tmp_block; @@ -676,7 +678,8 @@ private: } template - bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) { + bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) + { const auto in = checkAndGetColumn>(in_untyped); auto out = typeid_cast *>(out_untyped); if (!in || !out) @@ -688,9 +691,11 @@ private: } template - void executeImplNumToNum(const PaddedPODArray & src, PaddedPODArray & dst, const Array & boundaries) { + void executeImplNumToNum(const PaddedPODArray & src, PaddedPODArray & dst, const Array & boundaries) + { PaddedPODArray bvalues(boundaries.size()); - for (size_t i = 0; i < boundaries.size(); ++i) { + for (size_t i = 0; i < boundaries.size(); ++i) + { bvalues[i] = boundaries[i].get(); } From 57c5dbcd2b4f313b81e8f36ad964dd6b4d7c7bb7 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 5 Dec 2018 18:48:06 +0300 Subject: [PATCH 018/181] Code clean-up --- dbms/src/Storages/StorageDistributed.cpp | 4 +-- dbms/src/Storages/StorageDistributed.h | 2 ++ .../StorageDistributedShardsOptimizer.cpp | 24 +++++++----------- .../StorageDistributedShardsOptimizer.h | 25 ------------------- 4 files changed, 12 insertions(+), 43 deletions(-) delete mode 100644 dbms/src/Storages/StorageDistributedShardsOptimizer.h diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 0c9b17625ab..ad4e7fbe7da 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include #include @@ -270,8 +269,7 @@ BlockInputStreams StorageDistributed::read( if (settings.distributed_optimize_skip_select_on_unused_shards) { - auto optimizer = StorageDistributedShardsOptimizer(); - auto smaller_cluster = optimizer.skipUnusedShards(cluster, query_info, sharding_key_expr, sharding_key_column_name); + auto smaller_cluster = skipUnusedShards(cluster, query_info); if (smaller_cluster) cluster = smaller_cluster; diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index 1ae53f5637c..e14d9f7081f 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -166,6 +166,8 @@ protected: const ASTPtr & sharding_key_, const String & data_path_, bool attach); + + ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info); }; } diff --git a/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp b/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp index 10b282e896f..6ddbb6af525 100644 --- a/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp +++ b/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp @@ -1,23 +1,15 @@ -#include -#include +#include + #include #include -#include -#include #include #include -#include -#include #include #include #include -#include - -#include - namespace DB { namespace ErrorCodes @@ -25,6 +17,8 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; } +namespace +{ /// Contains a list of columns for conjunction: columns[0] AND columns[1] AND ... struct Conjunction { @@ -37,8 +31,8 @@ struct Disjunction std::vector conjunctions; }; -using DisjunctionsPtr = std::shared_ptr>; using Disjunctions = std::vector; +using DisjunctionsPtr = std::shared_ptr; static constexpr auto and_function_name = "and"; static constexpr auto equals_function_name = "equals"; @@ -313,12 +307,11 @@ bool hasRequiredColumns(const Block & block, ExpressionActionsPtr sharding_key_e return true; } -StorageDistributedShardsOptimizer::StorageDistributedShardsOptimizer() {} +} /** Returns a new cluster with fewer shards if constant folding for sharding_key_expr is possible * using constraints from WHERE condition, otherwise, returns nullptr. */ -ClusterPtr StorageDistributedShardsOptimizer::skipUnusedShards( - ClusterPtr cluster, const SelectQueryInfo & query_info, ExpressionActionsPtr sharding_key_expr, std::string sharding_key_column_name) +ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info) { const auto & select = typeid_cast(*query_info.query); @@ -352,7 +345,7 @@ ClusterPtr StorageDistributedShardsOptimizer::skipUnusedShards( sharding_key_expr->execute(block); if (!block || block.rows() != 1 || !block.has(sharding_key_column_name)) - throw Exception("Logical error: sharding_key_expr should evaluate as 1 row"); + throw Exception("Logical error: sharding_key_expr should evaluate as 1 row", ErrorCodes::TYPE_MISMATCH); const auto result = block.getByName(sharding_key_column_name); const auto selector = createSelector(cluster, result); @@ -363,4 +356,5 @@ ClusterPtr StorageDistributedShardsOptimizer::skipUnusedShards( return cluster->getClusterWithMultipleShards({shards.begin(), shards.end()}); } + } diff --git a/dbms/src/Storages/StorageDistributedShardsOptimizer.h b/dbms/src/Storages/StorageDistributedShardsOptimizer.h deleted file mode 100644 index f6dee543b71..00000000000 --- a/dbms/src/Storages/StorageDistributedShardsOptimizer.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include - -#include - -namespace Poco -{ -class Logger; -} - -namespace DB -{ -class StorageDistributedShardsOptimizer : private boost::noncopyable -{ -public: - StorageDistributedShardsOptimizer(); - ClusterPtr skipUnusedShards(ClusterPtr cluster, - const SelectQueryInfo & query_info, - ExpressionActionsPtr sharding_key_expr, - std::string sharding_key_column_name); -}; -} From 51b6dd67b06d9cc0c3cc09b7de4f45aaff3a1aad Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Fri, 7 Dec 2018 04:40:35 +0300 Subject: [PATCH 019/181] Match the process' effective user id against the data owner at the server startup. --- .../__init__.py | 0 .../configs/config.xml | 11 ++++++ .../configs/no_such_directory.xml | 3 ++ .../configs/owner_mismatch.xml | 3 ++ .../configs/users.xml | 3 ++ .../test.py | 37 +++++++++++++++++++ libs/libdaemon/src/BaseDaemon.cpp | 34 +++++++++++++++++ 7 files changed, 91 insertions(+) create mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/__init__.py create mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml create mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml create mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml create mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml create mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/test.py diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/__init__.py b/dbms/tests/integration/test_match_process_uid_against_data_owner/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml new file mode 100644 index 00000000000..48aa82349d3 --- /dev/null +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml @@ -0,0 +1,11 @@ + + + + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/clickhouse-server.err.log + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + + users.xml + diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml new file mode 100644 index 00000000000..80ddf7c4722 --- /dev/null +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml @@ -0,0 +1,3 @@ + + /no_such_directory/data/ + diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml new file mode 100644 index 00000000000..46d2dcc49ee --- /dev/null +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml @@ -0,0 +1,3 @@ + + /root/data/ + diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py b/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py new file mode 100644 index 00000000000..56c40530c36 --- /dev/null +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -0,0 +1,37 @@ +import os +import pwd +import pytest +import re + +from helpers.cluster import ClickHouseCluster + + +def expect_failure_with_message(config, expected_message): + cluster = ClickHouseCluster(__file__) + node = cluster.add_instance('node', main_configs=[config], with_zookeeper=False) + + with pytest.raises(Exception): + cluster.start() + + cluster.shutdown() # cleanup + + with open(os.path.join(node.path, 'logs/stderr.log')) as log: + last_message = log.readlines()[-1].strip() + if re.search(expected_message, last_message) is None: + pytest.fail('Expected the server to fail with a message "{}", but the last message is "{}"'.format(expected_message, last_message)) + + +def test_no_such_directory(): + expect_failure_with_message('configs/no_such_directory.xml', 'Failed to stat.*no_such_directory') + + +def test_different_user(): + current_user_id = os.getuid() + + if current_user_id != 0: + current_user_name = pwd.getpwuid(current_user_id).pw_name + + expect_failure_with_message( + 'configs/owner_mismatch.xml', + 'Effective user of the process \(({}|{})\) does not match the owner of the data \((0|root)\)'.format(current_user_id, current_user_name), + ) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index bad38c78529..ff06735e04e 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #if USE_UNWIND @@ -577,6 +578,17 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path } +static std::string getUserName(uid_t userId) { + /// Try to convert user id into user name. + const struct passwd * result = getpwuid(userId); + if (errno) + throw Poco::SystemException("Failed to get user name for " + DB::toString(userId)); + else if (result) + return result->pw_name; + return DB::toString(userId); +} + + void BaseDaemon::reloadConfiguration() { /** If the program is not run in daemon mode and 'config-file' is not specified, @@ -898,6 +910,28 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); } + std::string path = config().getString("path", DBMS_DEFAULT_PATH); + + /// Check that the process' user id matches the owner of the data. + const auto effectiveUserId = geteuid(); + struct stat statbuf; + if (stat(path.c_str(), &statbuf)) { + const auto parent = Poco::Path(path).parent().toString(); + if (stat(parent.c_str(), &statbuf)) { + throw Poco::SystemException("Failed to stat data path " + parent); + } + } + if (effectiveUserId != statbuf.st_uid) + { + const auto effectiveUser = getUserName(effectiveUserId); + const auto dataOwner = getUserName(statbuf.st_uid); + std::string message = "Effective user of the process (" + effectiveUser + + ") does not match the owner of the data (" + dataOwner + ")."; + if (effectiveUserId == 0) + message += " Run under 'sudo -u " + dataOwner + "'."; + throw Poco::SystemException(message); + } + DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); /// Write core dump on crash. From 42de08f3b92671b1c9609f8620a4123201933a03 Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Mon, 10 Dec 2018 01:15:59 +0300 Subject: [PATCH 020/181] Move euid check back to Server.cpp. Use getpwnam_r instead of getpwnam. Fix style. --- dbms/programs/server/Server.cpp | 64 +++++++++++++++++++++++-------- dbms/src/Common/ErrorCodes.cpp | 3 ++ libs/libdaemon/src/BaseDaemon.cpp | 34 ---------------- 3 files changed, 52 insertions(+), 49 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index f965bf58eaa..c7e44b60f56 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -2,7 +2,11 @@ #include #include +#include +#include #include +#include +#include #include #include #include @@ -70,6 +74,9 @@ namespace ErrorCodes extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int INVALID_CONFIG_PARAMETER; extern const int SYSTEM_ERROR; + extern const int FAILED_TO_STAT_DATA; + extern const int FAILED_TO_GETPWUID; + extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA; } @@ -83,6 +90,25 @@ static std::string getCanonicalPath(std::string && path) return std::move(path); } +static std::string getUserName(uid_t user_id) { + /// Try to convert user id into user name. + auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX); + if (buffer_size <= 0) + buffer_size = 32; + std::string buffer; + buffer.reserve(buffer_size); + + struct passwd passwd_entry; + struct passwd * result = nullptr; + const auto error = getpwuid_r(user_id, &passwd_entry, buffer.data(), buffer_size, &result); + + if (error) + throwFromErrno("Failed to find user name for " + toString(user_id), ErrorCodes::FAILED_TO_GETPWUID, error); + else if (result) + return result->pw_name; + return toString(user_id); +} + void Server::uninitialize() { logger().information("shutting down"); @@ -166,6 +192,22 @@ int Server::main(const std::vector & /*args*/) std::string path = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH)); std::string default_database = config().getString("default_database", "default"); + /// Check that the process' user id matches the owner of the data. + const auto effective_user_id = geteuid(); + struct stat statbuf; + const auto effective_user = getUserName(effective_user_id); + LOG_INFO(log, "effective_user = " + effective_user); + if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) + { + const auto effective_user = getUserName(effective_user_id); + const auto data_owner = getUserName(statbuf.st_uid); + std::string message = "Effective user of the process (" + effective_user + + ") does not match the owner of the data (" + data_owner + ")."; + if (effective_user_id == 0) + message += " Run under 'sudo -u " + data_owner + "'."; + throw Exception(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA); + } + global_context->setPath(path); /// Create directories for 'path' and for default database, if not exist. @@ -376,21 +418,13 @@ int Server::main(const std::vector & /*args*/) format_schema_path.createDirectories(); LOG_INFO(log, "Loading metadata."); - try - { - loadMetadataSystem(*global_context); - /// After attaching system databases we can initialize system log. - global_context->initializeSystemLogs(); - /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) - attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper); - /// Then, load remaining databases - loadMetadata(*global_context); - } - catch (...) - { - tryLogCurrentException(log, "Caught exception while loading metadata"); - throw; - } + loadMetadataSystem(*global_context); + /// After attaching system databases we can initialize system log. + global_context->initializeSystemLogs(); + /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) + attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper); + /// Then, load remaining databases + loadMetadata(*global_context); LOG_DEBUG(log, "Loaded metadata."); global_context->setCurrentDatabase(default_database); diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index e5b6028594b..f8d6b58eef7 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -402,6 +402,9 @@ namespace ErrorCodes extern const int SYSTEM_ERROR = 425; extern const int NULL_POINTER_DEREFERENCE = 426; extern const int CANNOT_COMPILE_REGEXP = 427; + extern const int FAILED_TO_STAT_DATA = 428; + extern const int FAILED_TO_GETPWUID = 429; + extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA = 430; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index ff06735e04e..bad38c78529 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #if USE_UNWIND @@ -578,17 +577,6 @@ static bool tryCreateDirectories(Poco::Logger * logger, const std::string & path } -static std::string getUserName(uid_t userId) { - /// Try to convert user id into user name. - const struct passwd * result = getpwuid(userId); - if (errno) - throw Poco::SystemException("Failed to get user name for " + DB::toString(userId)); - else if (result) - return result->pw_name; - return DB::toString(userId); -} - - void BaseDaemon::reloadConfiguration() { /** If the program is not run in daemon mode and 'config-file' is not specified, @@ -910,28 +898,6 @@ void BaseDaemon::initialize(Application & self) umask(umask_num); } - std::string path = config().getString("path", DBMS_DEFAULT_PATH); - - /// Check that the process' user id matches the owner of the data. - const auto effectiveUserId = geteuid(); - struct stat statbuf; - if (stat(path.c_str(), &statbuf)) { - const auto parent = Poco::Path(path).parent().toString(); - if (stat(parent.c_str(), &statbuf)) { - throw Poco::SystemException("Failed to stat data path " + parent); - } - } - if (effectiveUserId != statbuf.st_uid) - { - const auto effectiveUser = getUserName(effectiveUserId); - const auto dataOwner = getUserName(statbuf.st_uid); - std::string message = "Effective user of the process (" + effectiveUser + - ") does not match the owner of the data (" + dataOwner + ")."; - if (effectiveUserId == 0) - message += " Run under 'sudo -u " + dataOwner + "'."; - throw Poco::SystemException(message); - } - DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, ""); /// Write core dump on crash. From 93df4306af6e0777b28aed04f8e94ebfa24a1450 Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Mon, 10 Dec 2018 20:42:33 +0300 Subject: [PATCH 021/181] Fix style. --- dbms/programs/server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index c7e44b60f56..01d792fb8d2 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -90,7 +90,8 @@ static std::string getCanonicalPath(std::string && path) return std::move(path); } -static std::string getUserName(uid_t user_id) { +static std::string getUserName(uid_t user_id) +{ /// Try to convert user id into user name. auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX); if (buffer_size <= 0) From b56de46b7974a11c0a8129f3117146c9c9d2adc5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 10 Dec 2018 22:35:20 +0300 Subject: [PATCH 022/181] Update Server.cpp --- dbms/programs/server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index c7e44b60f56..01d792fb8d2 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -90,7 +90,8 @@ static std::string getCanonicalPath(std::string && path) return std::move(path); } -static std::string getUserName(uid_t user_id) { +static std::string getUserName(uid_t user_id) +{ /// Try to convert user id into user name. auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX); if (buffer_size <= 0) From df9833fd3f0cd4b35f73cc78c7e037728057b748 Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Tue, 11 Dec 2018 02:18:16 +0300 Subject: [PATCH 023/181] Remove no longer used error code. --- dbms/programs/server/Server.cpp | 1 - dbms/src/Common/ErrorCodes.cpp | 5 ++--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 01d792fb8d2..cd107a4b5e0 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -74,7 +74,6 @@ namespace ErrorCodes extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int INVALID_CONFIG_PARAMETER; extern const int SYSTEM_ERROR; - extern const int FAILED_TO_STAT_DATA; extern const int FAILED_TO_GETPWUID; extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA; } diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index f8d6b58eef7..0522382196a 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -402,9 +402,8 @@ namespace ErrorCodes extern const int SYSTEM_ERROR = 425; extern const int NULL_POINTER_DEREFERENCE = 426; extern const int CANNOT_COMPILE_REGEXP = 427; - extern const int FAILED_TO_STAT_DATA = 428; - extern const int FAILED_TO_GETPWUID = 429; - extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA = 430; + extern const int FAILED_TO_GETPWUID = 428; + extern const int MISMATCHING_USERS_FOR_PROCESS_AND_DATA = 429; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; From 30acd5000e6e22aa46f4fa302018f0afb05dd10b Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Tue, 11 Dec 2018 02:21:03 +0300 Subject: [PATCH 024/181] Fix bad merge. --- dbms/programs/server/Server.cpp | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index cd107a4b5e0..ad3de00cf46 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -418,13 +418,21 @@ int Server::main(const std::vector & /*args*/) format_schema_path.createDirectories(); LOG_INFO(log, "Loading metadata."); - loadMetadataSystem(*global_context); - /// After attaching system databases we can initialize system log. - global_context->initializeSystemLogs(); - /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) - attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper); - /// Then, load remaining databases - loadMetadata(*global_context); + try + { + loadMetadataSystem(*global_context); + /// After attaching system databases we can initialize system log. + global_context->initializeSystemLogs(); + /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) + attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper); + /// Then, load remaining databases + loadMetadata(*global_context); + } + catch (...) + { + tryLogCurrentException(log, "Caught exception while loading metadata"); + throw; + } LOG_DEBUG(log, "Loaded metadata."); global_context->setCurrentDatabase(default_database); From c9b984285d421b362b3cd3ab43f435b113e22022 Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Tue, 11 Dec 2018 02:32:21 +0300 Subject: [PATCH 025/181] Fail on user mismatch under root only. Just warn under non root user. --- dbms/programs/server/Server.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index ad3de00cf46..715e448c262 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -204,8 +204,14 @@ int Server::main(const std::vector & /*args*/) std::string message = "Effective user of the process (" + effective_user + ") does not match the owner of the data (" + data_owner + ")."; if (effective_user_id == 0) + { message += " Run under 'sudo -u " + data_owner + "'."; - throw Exception(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA); + throw Exception(message, ErrorCodes::MISMATCHING_USERS_FOR_PROCESS_AND_DATA); + } + else + { + LOG_WARNING(log, message); + } } global_context->setPath(path); From f9869b7d7ecb95c60aee4fffb7fc2ab30249a6cc Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Tue, 11 Dec 2018 02:36:20 +0300 Subject: [PATCH 026/181] Fix default buffer_size for getUserName(). Use real sysconf(_SC_GETPW_R_SIZE_MAX) value. --- dbms/programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 715e448c262..ecc61e125f1 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -94,7 +94,7 @@ static std::string getUserName(uid_t user_id) /// Try to convert user id into user name. auto buffer_size = sysconf(_SC_GETPW_R_SIZE_MAX); if (buffer_size <= 0) - buffer_size = 32; + buffer_size = 1024; std::string buffer; buffer.reserve(buffer_size); From ccdaf6a31b95c3e4f9ea63327cb3820efbaf82ea Mon Sep 17 00:00:00 2001 From: shangshujie Date: Tue, 11 Dec 2018 16:41:23 +0800 Subject: [PATCH 027/181] add java string hash, hive string hash, for compatible with java world --- dbms/src/Functions/FunctionsHashing.cpp | 2 + dbms/src/Functions/FunctionsHashing.h | 128 ++++++++++++++++++++++++ 2 files changed, 130 insertions(+) diff --git a/dbms/src/Functions/FunctionsHashing.cpp b/dbms/src/Functions/FunctionsHashing.cpp index bafd205e16c..edf3a66d515 100644 --- a/dbms/src/Functions/FunctionsHashing.cpp +++ b/dbms/src/Functions/FunctionsHashing.cpp @@ -20,6 +20,8 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 22c664d433e..cc0b3c4308c 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -956,6 +956,134 @@ private: } }; +struct JavaHashImpl +{ + static Int32 apply(const char * data, const size_t size) + { + Int32 h = 0; + for (int i = 0; i < (int)size; ++i) { + h = 31 * h + data[i]; + } + return h; + } +}; + +/* + * the java string hash implement, + * many system from java world use this string hash function or based it + */ +class FunctionJavaHash : public IFunction +{ +public: + static constexpr auto name = "JavaHash"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 1; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + const auto arg_count = arguments.size(); + if (arg_count != 1) + throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arg_count) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + const auto first_arg = arguments.front().get(); + if (!WhichDataType(first_arg).isString()) + throw Exception{"Illegal type " + first_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto arg_count = arguments.size(); + + if (arg_count == 1) + { + const auto col_untyped = block.getByPosition(arguments.front()).column.get(); + + if (const auto col_from = checkAndGetColumn(col_untyped)) + { + const auto size = col_from->size(); + auto col_to = ColumnInt32::create(size); + + const auto & chars = col_from->getChars(); + const auto & offsets = col_from->getOffsets(); + auto & out = col_to->getData(); + + ColumnString::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + out[i] = JavaHashImpl::apply( + reinterpret_cast(&chars[current_offset]), + offsets[i] - current_offset - 1); + + current_offset = offsets[i]; + } + + block.getByPosition(result).column = std::move(col_to); + } + else + throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + else + throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR}; + } +}; + +/* + * this hive function works for hive-version < 3.0, + * after 3.0, hive use murmur-hash3 + */ +class FunctionHiveHash : public FunctionJavaHash{ +public: + static constexpr auto name = "HiveHash"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const auto arg_count = arguments.size(); + + if (arg_count == 1) + { + const auto col_untyped = block.getByPosition(arguments.front()).column.get(); + + if (const auto col_from = checkAndGetColumn(col_untyped)) + { + const auto size = col_from->size(); + auto col_to = ColumnInt32::create(size); + + const auto & chars = col_from->getChars(); + const auto & offsets = col_from->getOffsets(); + auto & out = col_to->getData(); + + ColumnString::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + out[i] = JavaHashImpl::apply( + reinterpret_cast(&chars[current_offset]), + offsets[i] - current_offset - 1) & 0x7fffffff; + + current_offset = offsets[i]; + } + + block.getByPosition(result).column = std::move(col_to); + } + else + throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + else + throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR}; + } +}; struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; From 5de9577ecdd617de31337a83c688cf56f0a67569 Mon Sep 17 00:00:00 2001 From: shangshujie Date: Tue, 11 Dec 2018 18:29:59 +0800 Subject: [PATCH 028/181] add test function --- .../queries/0_stateless/00800_function_java_hash.reference | 4 ++++ dbms/tests/queries/0_stateless/00800_function_java_hash.sql | 4 ++++ 2 files changed, 8 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00800_function_java_hash.reference create mode 100644 dbms/tests/queries/0_stateless/00800_function_java_hash.sql diff --git a/dbms/tests/queries/0_stateless/00800_function_java_hash.reference b/dbms/tests/queries/0_stateless/00800_function_java_hash.reference new file mode 100644 index 00000000000..7f9c68298bd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00800_function_java_hash.reference @@ -0,0 +1,4 @@ +96354 +-676697544 +96354 +1470786104 diff --git a/dbms/tests/queries/0_stateless/00800_function_java_hash.sql b/dbms/tests/queries/0_stateless/00800_function_java_hash.sql new file mode 100644 index 00000000000..220c1f0ff60 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00800_function_java_hash.sql @@ -0,0 +1,4 @@ +select JavaHash('abc'); +select JavaHash('874293087'); +select HiveHash('abc'); +select HiveHash('874293087'); From 5de1b01d209bc11f86ef38f502909510ab1c903e Mon Sep 17 00:00:00 2001 From: shangshujie Date: Thu, 13 Dec 2018 12:47:00 +0800 Subject: [PATCH 029/181] fix indention --- dbms/src/Functions/FunctionsHashing.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index cc0b3c4308c..a6665de44f7 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -961,7 +961,8 @@ struct JavaHashImpl static Int32 apply(const char * data, const size_t size) { Int32 h = 0; - for (int i = 0; i < (int)size; ++i) { + for (int i = 0; i < (int)size; ++i) + { h = 31 * h + data[i]; } return h; @@ -1042,7 +1043,8 @@ public: * this hive function works for hive-version < 3.0, * after 3.0, hive use murmur-hash3 */ -class FunctionHiveHash : public FunctionJavaHash{ +class FunctionHiveHash : public FunctionJavaHash +{ public: static constexpr auto name = "HiveHash"; static FunctionPtr create(const Context &) { return std::make_shared(); } From 3209ac84d0c8043d7ecb039001e717c925dadde4 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 13 Dec 2018 12:34:12 +0300 Subject: [PATCH 030/181] Rewrite LEFT and RIGHT functions to corresponding SUBSTRING calls #3712 --- dbms/src/Parsers/ExpressionElementParsers.cpp | 112 ++++++++++++++++++ dbms/src/Parsers/ExpressionElementParsers.h | 14 +++ .../00765_sql_compatibility_aliases.reference | 2 + .../00765_sql_compatibility_aliases.sql | 2 + 4 files changed, 130 insertions(+) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 0912d2a5b7b..91d7cdd0470 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -388,6 +388,116 @@ bool ParserSubstringExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & e return true; } +bool ParserLeftExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + /// Rewrites left(expr, length) to SUBSTRING(expr, 1, length) + + ASTPtr expr_node; + ASTPtr start_node; + ASTPtr length_node; + + if (!ParserKeyword("LEFT").ignore(pos, expected)) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (!ParserExpression().parse(pos, expr_node, expected)) + return false; + + ParserToken(TokenType::Comma).ignore(pos, expected); + + if (!ParserExpression().parse(pos, length_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto expr_list_args = std::make_shared(); + start_node = std::make_shared(1); + expr_list_args->children = {expr_node, start_node, length_node}; + + auto func_node = std::make_shared(); + func_node->name = "substring"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + return true; +} + +bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + /// Rewrites RIGHT(expr, length) to substring(expr, greatest((length(expr) + 1) - length, 1)) + + ASTPtr expr_node; + ASTPtr length_node; + + if (!ParserKeyword("RIGHT").ignore(pos, expected)) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (!ParserExpression().parse(pos, expr_node, expected)) + return false; + + ParserToken(TokenType::Comma).ignore(pos, expected); + + if (!ParserExpression().parse(pos, length_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto length_expr_list_args = std::make_shared(); + length_expr_list_args->children = {expr_node}; + + auto length_func_node = std::make_shared(); + length_func_node->name = "length"; + length_func_node->arguments = std::move(length_expr_list_args); + length_func_node->children.push_back(length_func_node->arguments); + + auto plus_expr_list_args = std::make_shared(); + plus_expr_list_args->children = {length_func_node, std::make_shared(1)}; + + auto plus_node = std::make_shared(); + plus_node->name = "plus"; + plus_node->arguments = std::move(plus_expr_list_args); + plus_node->children.push_back(plus_node->arguments); + + auto minus_expr_list_args = std::make_shared(); + minus_expr_list_args->children = {plus_node, length_node}; + + auto minus_node = std::make_shared(); + minus_node->name = "minus"; + minus_node->arguments = std::move(minus_expr_list_args); + minus_node->children.push_back(minus_node->arguments); + + auto start_expr_list_args = std::make_shared(); + start_expr_list_args->children = {minus_node, std::make_shared(1)}; + + auto start_node = std::make_shared(); + start_node->name = "greatest"; + start_node->arguments = std::move(start_expr_list_args); + start_node->children.push_back(start_node->arguments); + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {expr_node, start_node}; + + auto func_node = std::make_shared(); + func_node->name = "substring"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + return true; +} + bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { auto begin = pos; @@ -751,6 +861,8 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserCastExpression().parse(pos, node, expected) || ParserExtractExpression().parse(pos, node, expected) || ParserSubstringExpression().parse(pos, node, expected) + || ParserLeftExpression().parse(pos, node, expected) + || ParserRightExpression().parse(pos, node, expected) || ParserCase().parse(pos, node, expected) || ParserFunction().parse(pos, node, expected) || ParserQualifiedAsterisk().parse(pos, node, expected) diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index a52864d97d1..39e20ac19df 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -103,6 +103,20 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserLeftExpression : public IParserBase +{ +protected: + const char * getName() const override { return "LEFT expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +class ParserRightExpression : public IParserBase +{ +protected: + const char * getName() const override { return "RIGHT expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + class ParserExtractExpression : public IParserBase { protected: diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index 7a70e443c1b..a739f4cf0e1 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -10,3 +10,5 @@ o 1 oo o +fo +r diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index 248514d134b..310e4e72e1f 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -12,3 +12,5 @@ select mid('foo', 3); select IF(3>2, 1, 0); select substring('foo' from 1 + 1); select SUBSTRING('foo' FROM 2 FOR 1); +select left('foo', 2); +select RIGHT('bar', 1); From e9cabc818d6798e015e72beb5b5680f853b810a7 Mon Sep 17 00:00:00 2001 From: shangshujie Date: Thu, 13 Dec 2018 18:36:31 +0800 Subject: [PATCH 031/181] fix indention --- dbms/src/Functions/FunctionsHashing.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index a6665de44f7..d65eebdeff3 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -961,8 +961,8 @@ struct JavaHashImpl static Int32 apply(const char * data, const size_t size) { Int32 h = 0; - for (int i = 0; i < (int)size; ++i) - { + for (int i = 0; i < (int)size; ++i) + { h = 31 * h + data[i]; } return h; From 034265f044c5e6033fe875d895ccf221c78c1a1c Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 13 Dec 2018 14:50:37 +0300 Subject: [PATCH 032/181] Introduce TRIM/LTRIM/RTRIM functions #3714 --- dbms/src/Parsers/ExpressionElementParsers.cpp | 149 ++++++++++++++++++ dbms/src/Parsers/ExpressionElementParsers.h | 7 + .../00765_sql_compatibility_aliases.reference | 5 + .../00765_sql_compatibility_aliases.sql | 5 + 4 files changed, 166 insertions(+) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 91d7cdd0470..babea27b3a3 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -388,6 +388,154 @@ bool ParserSubstringExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & e return true; } +bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + /// Handles all possible TRIM/LTRIM/RTRIM call variants + + std::string func_name = "replaceRegexpOne"; + bool trim_left = false; + bool trim_right = false; + bool char_override = false; + ASTPtr expr_node; + ASTPtr pattern_node; + ASTPtr to_remove; + + if (ParserKeyword("LTRIM").ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + trim_left = true; + } + else if (ParserKeyword("RTRIM").ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + trim_right = true; + } + else if (ParserKeyword("TRIM").ignore(pos, expected)) + { + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + if (ParserKeyword("BOTH").ignore(pos, expected)) + { + trim_left = true; + trim_right = true; + char_override = true; + } + else if (ParserKeyword("LEADING").ignore(pos, expected)) + { + trim_left = true; + char_override = true; + } + else if (ParserKeyword("TRAILING").ignore(pos, expected)) + { + trim_right = true; + char_override = true; + } + else + { + trim_left = true; + trim_right = true; + } + + if (char_override) + { + if (!ParserExpression().parse(pos, to_remove, expected)) /// TODO: wrap in RE2::QuoteMeta call + return false; + if (!ParserKeyword("FROM").ignore(pos, expected)) + return false; + } + } + + if (!(trim_left || trim_right)) + return false; + + if (!ParserExpression().parse(pos, expr_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + /// Convert to regexp replace function call + + auto pattern_func_node = std::make_shared(); + if (char_override) + { + auto pattern_list_args = std::make_shared(); + if (trim_left && trim_right) + { + pattern_list_args->children = { + std::make_shared("^["), + to_remove, + std::make_shared("]*|["), + to_remove, + std::make_shared("]*$") + }; + func_name = "replaceRegexpAll"; + } + else { + if (trim_left) + { + pattern_list_args->children = { + std::make_shared("^["), + to_remove, + std::make_shared("]*") + }; + } + else + { + /// trim_right == false not possible + pattern_list_args->children = { + std::make_shared("["), + to_remove, + std::make_shared("]*$") + }; + } + } + + pattern_func_node->name = "concat"; + pattern_func_node->arguments = std::move(pattern_list_args); + pattern_func_node->children.push_back(pattern_func_node->arguments); + + pattern_node = std::move(pattern_func_node); + } + else + { + if (trim_left && trim_right) + { + pattern_node = std::make_shared("^ *\\| *$"); + func_name = "replaceRegexpAll"; + } + else { + if (trim_left) + { + pattern_node = std::make_shared("^ *"); + } + else + { + /// trim_right == false not possible + pattern_node = std::make_shared(" *$"); + } + } + } + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {expr_node, pattern_node, std::make_shared("")}; + + auto func_node = std::make_shared(); + func_node->name = func_name; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + return true; +} + bool ParserLeftExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { /// Rewrites left(expr, length) to SUBSTRING(expr, 1, length) @@ -861,6 +1009,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserCastExpression().parse(pos, node, expected) || ParserExtractExpression().parse(pos, node, expected) || ParserSubstringExpression().parse(pos, node, expected) + || ParserTrimExpression().parse(pos, node, expected) || ParserLeftExpression().parse(pos, node, expected) || ParserRightExpression().parse(pos, node, expected) || ParserCase().parse(pos, node, expected) diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index 39e20ac19df..98aec4fa46c 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -103,6 +103,13 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserTrimExpression : public IParserBase +{ +protected: + const char * getName() const override { return "TRIM expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + class ParserLeftExpression : public IParserBase { protected: diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index a739f4cf0e1..1f1736ff552 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -12,3 +12,8 @@ oo o fo r +foo + foo +xxfoo +fooabba +fooabbafoo diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index 310e4e72e1f..e83f9f48de7 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -14,3 +14,8 @@ select substring('foo' from 1 + 1); select SUBSTRING('foo' FROM 2 FOR 1); select left('foo', 2); select RIGHT('bar', 1); +select ltrim(' foo'); +select RTRIM(' foo '); +select trim(TRAILING 'x' FROM 'xxfooxx'); +select Trim(LEADING 'ab' FROM 'abbafooabba'); +select TRIM(both 'ab' FROM 'abbafooabbafooabba'); From 180311b42b66c40594b36fa576ed32936e85aa11 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 13 Dec 2018 15:58:00 +0300 Subject: [PATCH 033/181] Introduce regexpQuoteMeta function to properly handle regexp special chars in TRIM #3714 --- dbms/src/Functions/FunctionRegexpQuoteMeta.h | 112 ++++++++++++++++++ dbms/src/Functions/regexpQuoteMeta.cpp | 10 ++ .../src/Functions/registerFunctionsString.cpp | 2 + dbms/src/Parsers/ExpressionElementParsers.cpp | 14 ++- .../00765_sql_compatibility_aliases.reference | 1 + .../00765_sql_compatibility_aliases.sql | 1 + 6 files changed, 138 insertions(+), 2 deletions(-) create mode 100644 dbms/src/Functions/FunctionRegexpQuoteMeta.h create mode 100644 dbms/src/Functions/regexpQuoteMeta.cpp diff --git a/dbms/src/Functions/FunctionRegexpQuoteMeta.h b/dbms/src/Functions/FunctionRegexpQuoteMeta.h new file mode 100644 index 00000000000..a4e2c0a1771 --- /dev/null +++ b/dbms/src/Functions/FunctionRegexpQuoteMeta.h @@ -0,0 +1,112 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#if USE_RE2_ST + #include // Y_IGNORE +#else + #define re2_st re2 +#endif + + +namespace DB +{ +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +class FunctionRegexpQuoteMeta : public IFunction +{ +public: + static constexpr auto name = "regexpQuoteMeta"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!WhichDataType(arguments[0].type).isString()) + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnPtr column_string = block.getByPosition(arguments[0]).column; + const ColumnString * input = checkAndGetColumn(column_string.get()); + + if (!input) + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + auto dst_column = ColumnString::create(); + auto & dst_data = dst_column->getChars(); + auto & dst_offsets = dst_column->getOffsets(); + + dst_data.resize(input->getChars().size() * input->size()); + dst_offsets.resize(input_rows_count); + + const ColumnString::Offsets & src_offsets = input->getOffsets(); + + auto source = reinterpret_cast(input->getChars().data()); + auto dst = reinterpret_cast(dst_data.data()); + auto dst_pos = dst; + + size_t src_offset_prev = 0; + + for (size_t row = 0; row < input_rows_count; ++row) + { + size_t srclen = src_offsets[row] - src_offset_prev - 1; + + re2_st::StringPiece unquoted(source, srclen); + const auto & quoted = re2_st::RE2::QuoteMeta(unquoted); + std::memcpy(dst_pos, quoted.data(), quoted.size()); + + source += srclen + 1; + dst_pos += quoted.size() + 1; + + dst_offsets[row] = dst_pos - dst; + src_offset_prev = src_offsets[row]; + } + + dst_data.resize(dst_pos - dst); + + block.getByPosition(result).column = std::move(dst_column); + } + +}; +} diff --git a/dbms/src/Functions/regexpQuoteMeta.cpp b/dbms/src/Functions/regexpQuoteMeta.cpp new file mode 100644 index 00000000000..a0d99fdefc6 --- /dev/null +++ b/dbms/src/Functions/regexpQuoteMeta.cpp @@ -0,0 +1,10 @@ +#include +#include + +namespace DB +{ +void registerFunctionRegexpQuoteMeta(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp index 3a07d8bbd65..d838ac9ff31 100644 --- a/dbms/src/Functions/registerFunctionsString.cpp +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -21,6 +21,7 @@ void registerFunctionSubstringUTF8(FunctionFactory &); void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); void registerFunctionStartsWith(FunctionFactory &); void registerFunctionEndsWith(FunctionFactory &); +void registerFunctionRegexpQuoteMeta(FunctionFactory &); #if USE_BASE64 void registerFunctionBase64Encode(FunctionFactory &); @@ -46,6 +47,7 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionAppendTrailingCharIfAbsent(factory); registerFunctionStartsWith(factory); registerFunctionEndsWith(factory); + registerFunctionRegexpQuoteMeta(factory); #if USE_BASE64 registerFunctionBase64Encode(factory); registerFunctionBase64Decode(factory); diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index babea27b3a3..43f39cd0079 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -444,10 +444,20 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect if (char_override) { - if (!ParserExpression().parse(pos, to_remove, expected)) /// TODO: wrap in RE2::QuoteMeta call + if (!ParserExpression().parse(pos, to_remove, expected)) return false; if (!ParserKeyword("FROM").ignore(pos, expected)) return false; + + auto quote_meta_func_node = std::make_shared(); + auto quote_meta_list_args = std::make_shared(); + quote_meta_list_args->children = {to_remove}; + + quote_meta_func_node->name = "regexpQuoteMeta"; + quote_meta_func_node->arguments = std::move(quote_meta_list_args); + quote_meta_func_node->children.push_back(quote_meta_func_node->arguments); + + to_remove = std::move(quote_meta_func_node); } } @@ -463,9 +473,9 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect /// Convert to regexp replace function call - auto pattern_func_node = std::make_shared(); if (char_override) { + auto pattern_func_node = std::make_shared(); auto pattern_list_args = std::make_shared(); if (trim_left && trim_right) { diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index 1f1736ff552..79c3925bde9 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -17,3 +17,4 @@ foo xxfoo fooabba fooabbafoo +foo* diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index e83f9f48de7..6cc77010fea 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -19,3 +19,4 @@ select RTRIM(' foo '); select trim(TRAILING 'x' FROM 'xxfooxx'); select Trim(LEADING 'ab' FROM 'abbafooabba'); select TRIM(both 'ab' FROM 'abbafooabbafooabba'); +select trim(LEADING '*[]{}|\\' FROM '\\|[[[}}}*foo*'); From 69f2667d63e90f5dd0ff0987834f96b501e6c853 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 13 Dec 2018 19:41:56 +0300 Subject: [PATCH 034/181] fix style errors --- dbms/src/Parsers/ExpressionElementParsers.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 43f39cd0079..55cc7fdcca9 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -488,7 +488,8 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect }; func_name = "replaceRegexpAll"; } - else { + else + { if (trim_left) { pattern_list_args->children = { @@ -521,7 +522,8 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect pattern_node = std::make_shared("^ *\\| *$"); func_name = "replaceRegexpAll"; } - else { + else + { if (trim_left) { pattern_node = std::make_shared("^ *"); From c8111704fb3e85b8c57d9b62757293293e422e64 Mon Sep 17 00:00:00 2001 From: shangshujie Date: Fri, 14 Dec 2018 14:12:23 +0800 Subject: [PATCH 035/181] add java hash code source url --- dbms/src/Functions/FunctionsHashing.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index d65eebdeff3..37fd9ae2093 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -956,6 +956,9 @@ private: } }; +/* + * http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452 + */ struct JavaHashImpl { static Int32 apply(const char * data, const size_t size) From 0753cfe81bf1095b8526ffa45c89e787ccd3c7d5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 14 Dec 2018 11:43:20 +0300 Subject: [PATCH 036/181] Merge FunctionRegexpQuoteMeta.h into regexpQuoteMeta.cpp + add comment about suboptimal library call --- dbms/src/Functions/FunctionRegexpQuoteMeta.h | 112 ------------------- dbms/src/Functions/regexpQuoteMeta.cpp | 109 +++++++++++++++++- 2 files changed, 108 insertions(+), 113 deletions(-) delete mode 100644 dbms/src/Functions/FunctionRegexpQuoteMeta.h diff --git a/dbms/src/Functions/FunctionRegexpQuoteMeta.h b/dbms/src/Functions/FunctionRegexpQuoteMeta.h deleted file mode 100644 index a4e2c0a1771..00000000000 --- a/dbms/src/Functions/FunctionRegexpQuoteMeta.h +++ /dev/null @@ -1,112 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#if USE_RE2_ST - #include // Y_IGNORE -#else - #define re2_st re2 -#endif - - -namespace DB -{ -using namespace GatherUtils; - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -class FunctionRegexpQuoteMeta : public IFunction -{ -public: - static constexpr auto name = "regexpQuoteMeta"; - - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if (!WhichDataType(arguments[0].type).isString()) - throw Exception( - "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const ColumnPtr column_string = block.getByPosition(arguments[0]).column; - const ColumnString * input = checkAndGetColumn(column_string.get()); - - if (!input) - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - auto dst_column = ColumnString::create(); - auto & dst_data = dst_column->getChars(); - auto & dst_offsets = dst_column->getOffsets(); - - dst_data.resize(input->getChars().size() * input->size()); - dst_offsets.resize(input_rows_count); - - const ColumnString::Offsets & src_offsets = input->getOffsets(); - - auto source = reinterpret_cast(input->getChars().data()); - auto dst = reinterpret_cast(dst_data.data()); - auto dst_pos = dst; - - size_t src_offset_prev = 0; - - for (size_t row = 0; row < input_rows_count; ++row) - { - size_t srclen = src_offsets[row] - src_offset_prev - 1; - - re2_st::StringPiece unquoted(source, srclen); - const auto & quoted = re2_st::RE2::QuoteMeta(unquoted); - std::memcpy(dst_pos, quoted.data(), quoted.size()); - - source += srclen + 1; - dst_pos += quoted.size() + 1; - - dst_offsets[row] = dst_pos - dst; - src_offset_prev = src_offsets[row]; - } - - dst_data.resize(dst_pos - dst); - - block.getByPosition(result).column = std::move(dst_column); - } - -}; -} diff --git a/dbms/src/Functions/regexpQuoteMeta.cpp b/dbms/src/Functions/regexpQuoteMeta.cpp index a0d99fdefc6..2e45a85376c 100644 --- a/dbms/src/Functions/regexpQuoteMeta.cpp +++ b/dbms/src/Functions/regexpQuoteMeta.cpp @@ -1,8 +1,115 @@ -#include +#include +#include +#include +#include #include +#include +#include +#include + +#include +#include + +#if USE_RE2_ST + #include // Y_IGNORE +#else + #define re2_st re2 +#endif namespace DB { +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +class FunctionRegexpQuoteMeta : public IFunction +{ +public: + static constexpr auto name = "regexpQuoteMeta"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!WhichDataType(arguments[0].type).isString()) + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnPtr column_string = block.getByPosition(arguments[0]).column; + const ColumnString * input = checkAndGetColumn(column_string.get()); + + if (!input) + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + auto dst_column = ColumnString::create(); + auto & dst_data = dst_column->getChars(); + auto & dst_offsets = dst_column->getOffsets(); + + dst_data.resize(input->getChars().size() * input->size()); + dst_offsets.resize(input_rows_count); + + const ColumnString::Offsets & src_offsets = input->getOffsets(); + + auto source = reinterpret_cast(input->getChars().data()); + auto dst = reinterpret_cast(dst_data.data()); + auto dst_pos = dst; + + size_t src_offset_prev = 0; + + for (size_t row = 0; row < input_rows_count; ++row) + { + size_t srclen = src_offsets[row] - src_offset_prev - 1; + + /// suboptimal, but uses original implementation from re2 + re2_st::StringPiece unquoted(source, srclen); + const auto & quoted = re2_st::RE2::QuoteMeta(unquoted); + std::memcpy(dst_pos, quoted.data(), quoted.size()); + + source += srclen + 1; + dst_pos += quoted.size() + 1; + + dst_offsets[row] = dst_pos - dst; + src_offset_prev = src_offsets[row]; + } + + dst_data.resize(dst_pos - dst); + + block.getByPosition(result).column = std::move(dst_column); + } + +}; + void registerFunctionRegexpQuoteMeta(FunctionFactory & factory) { factory.registerFunction(); From 7b4b768195d9a99bdb45538e28adb6c3ecb58fd7 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 14 Dec 2018 12:20:22 +0300 Subject: [PATCH 037/181] Some perf tests for trim functions --- dbms/tests/performance/trim/trim_numbers.xml | 34 ++++++++++++++++++ dbms/tests/performance/trim/trim_urls.xml | 38 ++++++++++++++++++++ 2 files changed, 72 insertions(+) create mode 100644 dbms/tests/performance/trim/trim_numbers.xml create mode 100644 dbms/tests/performance/trim/trim_urls.xml diff --git a/dbms/tests/performance/trim/trim_numbers.xml b/dbms/tests/performance/trim/trim_numbers.xml new file mode 100644 index 00000000000..07587c024ac --- /dev/null +++ b/dbms/tests/performance/trim/trim_numbers.xml @@ -0,0 +1,34 @@ + + trim_numbers + loop + + + + 10000 + + + 5000 + 20000 + + + + + + + + + + func + + trim( + ltrim( + rtrim( + trim(LEADING '012345' FROM + trim(TRAILING '012345' FROM + trim(BOTH '012345' FROM + + + + + SELECT count() FROM numbers(10000000) WHERE NOT ignore({func}toString(number))) + diff --git a/dbms/tests/performance/trim/trim_urls.xml b/dbms/tests/performance/trim/trim_urls.xml new file mode 100644 index 00000000000..3687068f086 --- /dev/null +++ b/dbms/tests/performance/trim/trim_urls.xml @@ -0,0 +1,38 @@ + + trim_urls + loop + + + hits_100m_single + + + + + 10000 + + + 5000 + 20000 + + + + + + + + + + func + + trim( + ltrim( + rtrim( + trim(LEADING 'htpsw:/' FROM + trim(TRAILING '/' FROM + trim(BOTH 'htpsw:/' FROM + + + + + SELECT count() FROM hits_100m_single WHERE NOT ignore({func}URL)) + From 649716bea1c0a6f2ecf8257332d22d8043affedc Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 14 Dec 2018 12:26:55 +0300 Subject: [PATCH 038/181] fix incorrect escaping --- dbms/src/Parsers/ExpressionElementParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 55cc7fdcca9..e2583bad3cd 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -519,7 +519,7 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect { if (trim_left && trim_right) { - pattern_node = std::make_shared("^ *\\| *$"); + pattern_node = std::make_shared("^ *| *$"); func_name = "replaceRegexpAll"; } else From ebc0a165f499e9ff6b267e4e91624f87aad7191d Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 14 Dec 2018 12:32:24 +0300 Subject: [PATCH 039/181] Add perf test for right function --- dbms/tests/performance/right/right.xml | 34 ++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 dbms/tests/performance/right/right.xml diff --git a/dbms/tests/performance/right/right.xml b/dbms/tests/performance/right/right.xml new file mode 100644 index 00000000000..7622210133f --- /dev/null +++ b/dbms/tests/performance/right/right.xml @@ -0,0 +1,34 @@ + + right + loop + + + hits_100m_single + + + + + 10000 + + + 5000 + 20000 + + + + + + + + + + func + + right(URL, 16) + substring(URL, greatest(minus(plus(length(URL), 1), 16), 1)) + + + + + SELECT count() FROM hits_100m_single WHERE NOT ignore({func}) + From 2279058f0dbdb9913dde99e9e68ef410537c4835 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 14 Dec 2018 14:42:49 +0300 Subject: [PATCH 040/181] Simplify right(expr, length) function to be just an alias to substring(expr, -length) --- dbms/src/Functions/right.cpp | 117 ++++++++++++++++++ dbms/src/Parsers/ExpressionElementParsers.cpp | 30 +---- 2 files changed, 120 insertions(+), 27 deletions(-) create mode 100644 dbms/src/Functions/right.cpp diff --git a/dbms/src/Functions/right.cpp b/dbms/src/Functions/right.cpp new file mode 100644 index 00000000000..2e45a85376c --- /dev/null +++ b/dbms/src/Functions/right.cpp @@ -0,0 +1,117 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#if USE_RE2_ST + #include // Y_IGNORE +#else + #define re2_st re2 +#endif + +namespace DB +{ +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +class FunctionRegexpQuoteMeta : public IFunction +{ +public: + static constexpr auto name = "regexpQuoteMeta"; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!WhichDataType(arguments[0].type).isString()) + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnPtr column_string = block.getByPosition(arguments[0]).column; + const ColumnString * input = checkAndGetColumn(column_string.get()); + + if (!input) + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + auto dst_column = ColumnString::create(); + auto & dst_data = dst_column->getChars(); + auto & dst_offsets = dst_column->getOffsets(); + + dst_data.resize(input->getChars().size() * input->size()); + dst_offsets.resize(input_rows_count); + + const ColumnString::Offsets & src_offsets = input->getOffsets(); + + auto source = reinterpret_cast(input->getChars().data()); + auto dst = reinterpret_cast(dst_data.data()); + auto dst_pos = dst; + + size_t src_offset_prev = 0; + + for (size_t row = 0; row < input_rows_count; ++row) + { + size_t srclen = src_offsets[row] - src_offset_prev - 1; + + /// suboptimal, but uses original implementation from re2 + re2_st::StringPiece unquoted(source, srclen); + const auto & quoted = re2_st::RE2::QuoteMeta(unquoted); + std::memcpy(dst_pos, quoted.data(), quoted.size()); + + source += srclen + 1; + dst_pos += quoted.size() + 1; + + dst_offsets[row] = dst_pos - dst; + src_offset_prev = src_offsets[row]; + } + + dst_data.resize(dst_pos - dst); + + block.getByPosition(result).column = std::move(dst_column); + } + +}; + +void registerFunctionRegexpQuoteMeta(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index e2583bad3cd..808be36b1f2 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -590,7 +590,7 @@ bool ParserLeftExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - /// Rewrites RIGHT(expr, length) to substring(expr, greatest((length(expr) + 1) - length, 1)) + /// Rewrites RIGHT(expr, length) to substring(expr, -length) ASTPtr expr_node; ASTPtr length_node; @@ -614,35 +614,11 @@ bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; ++pos; - auto length_expr_list_args = std::make_shared(); - length_expr_list_args->children = {expr_node}; - - auto length_func_node = std::make_shared(); - length_func_node->name = "length"; - length_func_node->arguments = std::move(length_expr_list_args); - length_func_node->children.push_back(length_func_node->arguments); - - auto plus_expr_list_args = std::make_shared(); - plus_expr_list_args->children = {length_func_node, std::make_shared(1)}; - - auto plus_node = std::make_shared(); - plus_node->name = "plus"; - plus_node->arguments = std::move(plus_expr_list_args); - plus_node->children.push_back(plus_node->arguments); - - auto minus_expr_list_args = std::make_shared(); - minus_expr_list_args->children = {plus_node, length_node}; - - auto minus_node = std::make_shared(); - minus_node->name = "minus"; - minus_node->arguments = std::move(minus_expr_list_args); - minus_node->children.push_back(minus_node->arguments); - auto start_expr_list_args = std::make_shared(); - start_expr_list_args->children = {minus_node, std::make_shared(1)}; + start_expr_list_args->children = {length_node}; auto start_node = std::make_shared(); - start_node->name = "greatest"; + start_node->name = "negate"; start_node->arguments = std::move(start_expr_list_args); start_node->children.push_back(start_node->arguments); From 9dca1e3ca2ee215ab8b4258bb3efbfa264084935 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 14 Dec 2018 14:45:45 +0300 Subject: [PATCH 041/181] Test for left/right with cut size larger than input --- .../0_stateless/00765_sql_compatibility_aliases.reference | 2 ++ .../queries/0_stateless/00765_sql_compatibility_aliases.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index 79c3925bde9..691d92957a4 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -11,7 +11,9 @@ o oo o fo +foo r +bar foo foo xxfoo diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index 6cc77010fea..f01560587a8 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -13,7 +13,9 @@ select IF(3>2, 1, 0); select substring('foo' from 1 + 1); select SUBSTRING('foo' FROM 2 FOR 1); select left('foo', 2); +select LEFT('foo', 123); select RIGHT('bar', 1); +select right('bar', 123); select ltrim(' foo'); select RTRIM(' foo '); select trim(TRAILING 'x' FROM 'xxfooxx'); From 6df757c6f714a866d722fbe0f3ca6a8e3174623d Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 11 Dec 2018 20:43:12 +0300 Subject: [PATCH 042/181] Refactor constant folding and make it reusable for primary_key_expr --- .gitignore | 3 + dbms/src/Core/Block.h | 4 +- .../evaluateConstantExpression.cpp | 256 +++++++++++++- .../Interpreters/evaluateConstantExpression.h | 6 + dbms/src/Storages/MergeTree/KeyCondition.cpp | 2 +- dbms/src/Storages/MergeTree/KeyCondition.h | 2 +- .../StorageDistributedShardsOptimizer.cpp | 320 ++---------------- ...ize_skip_select_on_unused_shards.reference | 19 +- ...d_optimize_skip_select_on_unused_shards.sh | 106 ++++++ ..._optimize_skip_select_on_unused_shards.sql | 17 - 10 files changed, 401 insertions(+), 334 deletions(-) create mode 100755 dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh delete mode 100644 dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql diff --git a/.gitignore b/.gitignore index 585a4074767..9816f1cbb6c 100644 --- a/.gitignore +++ b/.gitignore @@ -248,3 +248,6 @@ website/package-lock.json # Ignore files for locally disabled tests /dbms/tests/queries/**/*.disabled + +# cquery cache +/.cquery-cache diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 690f9720af1..71ef40bbabc 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -94,8 +94,8 @@ public: /// Approximate number of allocated bytes in memory - for profiling and limits. size_t allocatedBytes() const; - operator bool() const { return !data.empty(); } - bool operator!() const { return data.empty(); } + operator bool() const { return !!columns(); } + bool operator!() const { return !this->operator bool(); } /** Get a list of column names separated by commas. */ std::string dumpNames() const; diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index adb60d34eaa..8d422c248b1 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -1,18 +1,20 @@ -#include +#include + #include #include -#include -#include -#include -#include +#include #include #include -#include -#include +#include #include -#include -#include +#include +#include +#include +#include +#include +#include #include +#include namespace DB @@ -57,7 +59,7 @@ std::pair> evaluateConstantExpression(co ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & context) { - /// Branch with string in qery. + /// Branch with string in query. if (typeid_cast(node.get())) return node; @@ -77,4 +79,238 @@ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, cons return evaluateConstantExpressionAsLiteral(node, context); } +namespace +{ + using Conjunction = ColumnsWithTypeAndName; + using Disjunction = std::vector; + + Disjunction analyzeEquals(const ASTIdentifier * identifier, const ASTLiteral * literal, const ExpressionActionsPtr & expr) + { + if (!identifier || !literal) + { + return {}; + } + + for (const auto & name_and_type : expr->getRequiredColumnsWithTypes()) + { + const auto & name = name_and_type.name; + const auto & type = name_and_type.type; + + if (name == identifier->name) + { + ColumnWithTypeAndName column; + // FIXME: what to do if field is not convertable? + column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); + column.name = name; + column.type = type; + return {{std::move(column)}}; + } + } + + return {}; + } + + Disjunction andDNF(const Disjunction & left, const Disjunction & right) + { + if (left.empty()) + { + return right; + } + + Disjunction result; + + for (const auto & conjunct1 : left) + { + for (const auto & conjunct2 : right) + { + Conjunction new_conjunct{conjunct1}; + new_conjunct.insert(new_conjunct.end(), conjunct2.begin(), conjunct2.end()); + result.emplace_back(new_conjunct); + } + } + + return result; + } + + Disjunction analyzeFunction(const ASTFunction * fn, const ExpressionActionsPtr & expr) + { + if (!fn) + { + return {}; + } + + // TODO: enumerate all possible function names! + + if (fn->name == "equals") + { + const auto * left = fn->arguments->children.front().get(); + const auto * right = fn->arguments->children.back().get(); + const auto * identifier = typeid_cast(left) ? typeid_cast(left) + : typeid_cast(right); + const auto * literal = typeid_cast(left) ? typeid_cast(left) + : typeid_cast(right); + + return analyzeEquals(identifier, literal, expr); + } + else if (fn->name == "in") + { + const auto * left = fn->arguments->children.front().get(); + const auto * right = fn->arguments->children.back().get(); + const auto * identifier = typeid_cast(left); + const auto * inner_fn = typeid_cast(right); + + if (!inner_fn) + { + return {}; + } + + const auto * tuple = typeid_cast(inner_fn->children.front().get()); + + if (!tuple) + { + return {}; + } + + Disjunction result; + + for (const auto & child : tuple->children) + { + const auto * literal = typeid_cast(child.get()); + const auto dnf = analyzeEquals(identifier, literal, expr); + + if (dnf.empty()) + { + return {}; + } + + result.insert(result.end(), dnf.begin(), dnf.end()); + } + + return result; + } + else if (fn->name == "or") + { + const auto * args = typeid_cast(fn->children.front().get()); + + if (!args) + { + return {}; + } + + Disjunction result; + + for (const auto & arg : args->children) + { + const auto dnf = analyzeFunction(typeid_cast(arg.get()), expr); + + if (dnf.empty()) + { + return {}; + } + + result.insert(result.end(), dnf.begin(), dnf.end()); + } + + return result; + } + else if (fn->name == "and") + { + const auto * args = typeid_cast(fn->children.front().get()); + + if (!args) + { + return {}; + } + + Disjunction result; + + for (const auto & arg : args->children) + { + const auto dnf = analyzeFunction(typeid_cast(arg.get()), expr); + + if (dnf.empty()) + { + continue; + } + + result = andDNF(result, dnf); + } + + return result; + } + + return {}; + } +} + +// TODO: distinguish always-false and failed evaluation results, +// assume failed if returned empty `Blocks` for now. +Blocks evaluateConstantExpressionAsBlock(const ASTPtr & node, const ExpressionActionsPtr & target_expr) +{ + Blocks result; + + // TODO: `node` may be always-false literal. + + if (const auto fn = typeid_cast(node.get())) + { + const auto dnf = analyzeFunction(fn, target_expr); + + if (dnf.empty()) + { + return result; + } + + auto hasRequiredColumns = [&target_expr](const Block & block) -> bool + { + for (const auto & name : target_expr->getRequiredColumns()) + { + bool hasColumn = false; + for (const auto & column_name : block.getNames()) + { + if (column_name == name) + { + hasColumn = true; + break; + } + } + + if (!hasColumn) + return false; + } + + return true; + }; + + for (const auto & conjunct : dnf) + { + Block block(conjunct); + + // Block should contain all required columns from `target_expr` + if (!hasRequiredColumns(block)) + { + return {}; + } + + target_expr->execute(block); + + if (block.rows() == 1) + { + result.push_back(block); + } + else if (block.rows() == 0) + { + // filter out cases like "WHERE a = 1 AND a = 2" + continue; + } + else + { + // FIXME: shouldn't happen + return {}; + } + } + } + + return result; +} + } diff --git a/dbms/src/Interpreters/evaluateConstantExpression.h b/dbms/src/Interpreters/evaluateConstantExpression.h index c35b7177622..e13a816149a 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.h +++ b/dbms/src/Interpreters/evaluateConstantExpression.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -10,8 +11,10 @@ namespace DB { class Context; +class ExpressionActions; class IDataType; +using ExpressionActionsPtr = std::shared_ptr; /** Evaluate constant expression and its type. * Used in rare cases - for elements of set for IN, for data to INSERT. @@ -33,4 +36,7 @@ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & */ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const Context & context); +// FIXME: collapse returned blocks into a single block. +Blocks evaluateConstantExpressionAsBlock(const ASTPtr & node, const ExpressionActionsPtr & target_expr); + } diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index 9484bd8c3cc..31a4e08707f 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -313,7 +313,7 @@ bool KeyCondition::addCondition(const String & column, const Range & range) return true; } -/** Computes value of constant expression and it data type. +/** Computes value of constant expression and its data type. * Returns false, if expression isn't constant. */ static bool getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type) diff --git a/dbms/src/Storages/MergeTree/KeyCondition.h b/dbms/src/Storages/MergeTree/KeyCondition.h index d025f70bf09..1d700ad80d9 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.h +++ b/dbms/src/Storages/MergeTree/KeyCondition.h @@ -253,7 +253,7 @@ public: /// Get the maximum number of the key element used in the condition. size_t getMaxKeyColumn() const; - /// Impose an additional condition: the value in the column column must be in the `range` range. + /// Impose an additional condition: the value in the column `column` must be in the range `range`. /// Returns whether there is such a column in the key. bool addCondition(const String & column, const Range & range); diff --git a/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp b/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp index 6ddbb6af525..ccdcfea2faa 100644 --- a/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp +++ b/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp @@ -1,266 +1,23 @@ #include #include -#include - -#include #include - -#include -#include +#include #include +#include namespace DB { + namespace ErrorCodes { - extern const int TYPE_MISMATCH; + +extern const int TYPE_MISMATCH; + } namespace { -/// Contains a list of columns for conjunction: columns[0] AND columns[1] AND ... -struct Conjunction -{ - ColumnsWithTypeAndName columns; -}; - -/// Contains a list of disjunctions: disjunctions[0] OR disjunctions[1] OR ... -struct Disjunction -{ - std::vector conjunctions; -}; - -using Disjunctions = std::vector; -using DisjunctionsPtr = std::shared_ptr; - -static constexpr auto and_function_name = "and"; -static constexpr auto equals_function_name = "equals"; -static constexpr auto in_function_name = "in"; -static constexpr auto or_function_name = "or"; -static constexpr auto tuple_function_name = "tuple"; - -void logDebug(std::string message) -{ - LOG_DEBUG(&Logger::get("(StorageDistributedShardsOptimizer)"), message); -} - -/// Returns disjunction equivalent to `disjunctions AND another`. -Disjunctions pairwiseAnd(const Disjunctions & disjunctions, const Disjunctions & another) -{ - Disjunctions new_disjunctions; - - if (disjunctions.empty()) - { - return another; - } - - if (another.empty()) - { - return disjunctions; - } - - for (const auto disjunction : disjunctions) - { - for (const auto another_disjunction : another) - { - std::vector new_conjunctions; - - for (const auto conjunction : disjunction.conjunctions) - { - for (const auto another_conjunction : another_disjunction.conjunctions) - { - ColumnsWithTypeAndName new_columns; - new_columns.insert(std::end(new_columns), conjunction.columns.begin(), conjunction.columns.end()); - new_columns.insert(std::end(new_columns), another_conjunction.columns.begin(), another_conjunction.columns.end()); - - new_conjunctions.push_back(Conjunction{new_columns}); - } - } - - new_disjunctions.push_back(Disjunction{new_conjunctions}); - } - } - - return new_disjunctions; -} - -/// Given `ident = literal` expr, returns disjunctions relevant for constant folding in sharding_key_expr. -DisjunctionsPtr analyzeEquals(const ASTIdentifier * ident, const ASTLiteral * literal, ExpressionActionsPtr sharding_key_expr) -{ - for (const auto name_and_type : sharding_key_expr->getRequiredColumnsWithTypes()) - { - const auto type = name_and_type.type; - const auto name = name_and_type.name; - - if (name == ident->name) - { - ColumnWithTypeAndName column; - - column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); - column.type = type; - column.name = name; - - const auto columns = ColumnsWithTypeAndName{column}; - const auto conjunction = Conjunction{columns}; - const auto disjunction = Disjunction{{conjunction}}; - const Disjunctions disjunctions = {disjunction}; - - return std::make_shared(disjunctions); - } - } - - const Disjunctions disjunctions = {}; - return std::make_shared(disjunctions); -} - -/// Given `ident IN (..literals)` expr, returns disjunctions relevant for constant folding in sharding_key_expr. -DisjunctionsPtr analyzeIn( - const ASTIdentifier * ident, const std::vector literals, ExpressionActionsPtr sharding_key_expr) -{ - Disjunctions disjunctions; - - for (const auto literal : literals) - { - const auto inner_disjunctions = analyzeEquals(ident, literal, sharding_key_expr); - - if (!inner_disjunctions) - return nullptr; - - disjunctions.insert(std::end(disjunctions), inner_disjunctions->begin(), inner_disjunctions->end()); - } - - return std::make_shared(disjunctions); -} - -/// Given WHERE condition, returns disjunctions relevant for constant folding in sharding_key_expr. -DisjunctionsPtr analyzeQuery(const ASTFunction * function, ExpressionActionsPtr sharding_key_expr) -{ - if (function->name == equals_function_name) - { - auto left_arg = function->arguments->children.front().get(); - auto right_arg = function->arguments->children.back().get(); - - // try to ensure left_arg points to ASTIdentifier - if (!typeid_cast(left_arg) && typeid_cast(right_arg)) - std::swap(left_arg, right_arg); - - const auto ident = typeid_cast(left_arg); - const auto literal = typeid_cast(right_arg); - - if (!ident || !literal) - { - logDebug("didn't match pattern ident = "); - return nullptr; - } - - return analyzeEquals(ident, literal, sharding_key_expr); - } - else if (function->name == in_function_name) - { - const auto left_arg = function->arguments->children.front().get(); - const auto right_arg = function->arguments->children.back().get(); - - const auto ident = typeid_cast(left_arg); - const auto inner_function = typeid_cast(right_arg); - - if (!ident || !inner_function || inner_function->name != tuple_function_name) - { - logDebug("didn't match pattern ident IN tuple(...)"); - return nullptr; - } - - std::vector literals; - const auto expr_list = typeid_cast(inner_function->children.front().get()); - - if (!expr_list) - { - logDebug("expected ExpressionList in tuple, got: " + inner_function->getID()); - return nullptr; - } - - for (const auto child : expr_list->children) - { - if (const auto child_literal = typeid_cast(child.get())) - { - literals.push_back(child_literal); - } - else - { - logDebug("non-literal in IN expression, got: " + child->getID()); - return nullptr; - } - } - - return analyzeIn(ident, literals, sharding_key_expr); - } - else if (function->name == or_function_name) - { - const auto expr_list = typeid_cast(function->children.front().get()); - - if (!expr_list) - { - logDebug("expected ExpressionList in IN, got: " + function->getID()); - return nullptr; - } - - Disjunctions disjunctions; - - for (const auto child : expr_list->children) - { - // we can't ignore expr we can't analyze because it can widden the set of shards - if (const auto child_function = typeid_cast(child.get())) - { - const auto child_disjunctions = analyzeQuery(child_function, sharding_key_expr); - - if (!child_disjunctions) - return nullptr; - - disjunctions.insert(std::end(disjunctions), child_disjunctions->begin(), child_disjunctions->end()); - } - else - { - logDebug("non-function expression in OR, got: " + child->getID()); - return nullptr; - } - } - - return std::make_shared(disjunctions); - } - else if (function->name == and_function_name) - { - const auto expr_list = typeid_cast(function->children.front().get()); - - if (!expr_list) - { - logDebug("expected ExpressionList in AND, got: " + function->getID()); - return nullptr; - } - - Disjunctions disjunctions; - - for (const auto child : expr_list->children) - { - // we can skip everything we can't analyze because it can only narrow the set of shards - if (const auto child_function = typeid_cast(child.get())) - { - const auto child_disjunctions = analyzeQuery(child_function, sharding_key_expr); - - if (!child_disjunctions) - continue; - - disjunctions = pairwiseAnd(disjunctions, *child_disjunctions); - } - } - - return std::make_shared(disjunctions); - } - else - { - logDebug("unsupported function: " + function->name); - return nullptr; - } -} /// the same as DistributedBlockOutputStream::createSelector, should it be static? IColumn::Selector createSelector(const ClusterPtr cluster, const ColumnWithTypeAndName & result) @@ -285,73 +42,38 @@ IColumn::Selector createSelector(const ClusterPtr cluster, const ColumnWithTypeA throw Exception{"Sharding key expression does not evaluate to an integer type", ErrorCodes::TYPE_MISMATCH}; } -/// Returns true if block has all columns required by sharding_key_expr. -bool hasRequiredColumns(const Block & block, ExpressionActionsPtr sharding_key_expr) -{ - for (const auto name : sharding_key_expr->getRequiredColumns()) - { - bool hasColumn = false; - for (const auto column_name : block.getNames()) - { - if (column_name == name) - { - hasColumn = true; - break; - } - } - - if (!hasColumn) - return false; - } - - return true; } -} - -/** Returns a new cluster with fewer shards if constant folding for sharding_key_expr is possible - * using constraints from WHERE condition, otherwise, returns nullptr. */ +/// Returns a new cluster with fewer shards if constant folding for `sharding_key_expr` is possible +/// using constraints from "WHERE" condition, otherwise returns `nullptr` ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info) { const auto & select = typeid_cast(*query_info.query); if (!select.where_expression) + { return nullptr; + } - const auto function = typeid_cast(select.where_expression.get()); + const auto blocks = evaluateConstantExpressionAsBlock(select.where_expression, sharding_key_expr); - if (!function) - return nullptr; - - const auto disjunctions = analyzeQuery(function, sharding_key_expr); - - // didn't get definite answer from analysis, about optimization - if (!disjunctions) + // Can't get definite answer if we can skip any shards + if (blocks.empty()) + { return nullptr; + } std::set shards; - for (const auto disjunction : *disjunctions) + for (const auto & block : blocks) { - for (const auto conjunction : disjunction.conjunctions) - { - Block block(conjunction.columns); + if (!block.has(sharding_key_column_name)) + throw Exception("sharding_key_expr should evaluate as a single row", ErrorCodes::TYPE_MISMATCH); - // check if sharding_key_expr requires column that we don't know anything about - // if so, we don't have enough information to optimize - if (!hasRequiredColumns(block, sharding_key_expr)) - return nullptr; + const auto result = block.getByName(sharding_key_column_name); + const auto selector = createSelector(cluster, result); - sharding_key_expr->execute(block); - - if (!block || block.rows() != 1 || !block.has(sharding_key_column_name)) - throw Exception("Logical error: sharding_key_expr should evaluate as 1 row", ErrorCodes::TYPE_MISMATCH); - - const auto result = block.getByName(sharding_key_column_name); - const auto selector = createSelector(cluster, result); - - shards.insert(selector.begin(), selector.end()); - } + shards.insert(selector.begin(), selector.end()); } return cluster->getClusterWithMultipleShards({shards.begin(), shards.end()}); diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference index 44e0be8e356..3161be13b12 100644 --- a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference +++ b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference @@ -1,4 +1,15 @@ -0 -0 -0 -0 +OK +OK +1 +OK +4 +2 +1 +1 +1 +4 +OK +OK +OK +OK +OK diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh new file mode 100755 index 00000000000..84fec6de872 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh @@ -0,0 +1,106 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.mergetree;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test.distributed;" + +${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.mergetree (a Int64, b Int64, c Int64) ENGINE = MergeTree ORDER BY (a, b);" +${CLICKHOUSE_CLIENT} --query "CREATE TABLE test.distributed AS test.mergetree ENGINE = Distributed(test_unavailable_shard, test, mergetree, jumpConsistentHash(a+b, 2));" + +${CLICKHOUSE_CLIENT} --query "INSERT INTO test.mergetree VALUES (0, 0, 0);" +${CLICKHOUSE_CLIENT} --query "INSERT INTO test.mergetree VALUES (1, 0, 0);" +${CLICKHOUSE_CLIENT} --query "INSERT INTO test.mergetree VALUES (0, 1, 1);" +${CLICKHOUSE_CLIENT} --query "INSERT INTO test.mergetree VALUES (1, 1, 1);" + +# Should fail because second shard is unavailable +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM test.distributed;" 2>&1 \ +| fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' + +# Should fail without setting `distributed_optimize_skip_select_on_unused_shards` +${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0;" 2>&1 \ +| fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' + +# Should pass now +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0; +" + +# Should still fail because of matching unavailable shard +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 2 AND b = 2; +" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' + +# Try more complext expressions for constant folding - all should pass. + +# TODO: should pass one day. +#${CLICKHOUSE_CLIENT} -n --query=" +# SET distributed_optimize_skip_select_on_unused_shards = 1; +# SELECT count(*) FROM test.distributed WHERE a = 1 AND a = 0; +#" + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a IN (0, 1) AND b IN (0, 1); +" + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 OR a = 1 AND b = 1; +" + +# TODO: should pass one day. +#${CLICKHOUSE_CLIENT} -n --query=" +# SET distributed_optimize_skip_select_on_unused_shards = 1; +# SELECT count(*) FROM test.distributed WHERE a = 0 AND b >= 0 AND b <= 1; +#" + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 AND c = 0; +" + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 AND c != 10; +" + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 AND (a+b)*b != 12; +" + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE (a = 0 OR a = 1) AND (b = 0 OR b = 1); +" + +# These ones should fail. + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b <= 1; +" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND c = 0; +" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 OR a = 1 AND b = 0; +" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 OR a = 2 AND b = 2; +" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' + +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 OR c = 0; +" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql deleted file mode 100644 index 13f88ead4b5..00000000000 --- a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sql +++ /dev/null @@ -1,17 +0,0 @@ -SET distributed_optimize_skip_select_on_unused_shards = 1; - -DROP TABLE IF EXISTS test.mergetree; -DROP TABLE IF EXISTS test.distributed; - -CREATE TABLE test.mergetree (a Int64, b Int64) ENGINE = MergeTree ORDER BY (a, b); -CREATE TABLE test.distributed AS test.mergetree ENGINE = Distributed(test_unavailable_shard, test, mergetree, jumpConsistentHash(a+b, 2)); - -INSERT INTO test.mergetree VALUES (0, 0); -INSERT INTO test.mergetree VALUES (1, 0); -INSERT INTO test.mergetree VALUES (0, 1); -INSERT INTO test.mergetree VALUES (1, 1); - -/* without setting, quering of the second shard will fail because it isn't available */ - -SELECT jumpConsistentHash(a+b, 2) FROM test.distributed -WHERE (a+b > 0 AND a = 0 AND b = 0) OR (a IN (0, 1) AND b IN (0, 1)) OR ((a = 1 OR a = 1) AND b = 2); From 8c8dc333e5ce2ead51499002bb5a371fd12e3b1f Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 17 Dec 2018 21:06:50 +0300 Subject: [PATCH 043/181] Fix unavailable shard's host:port for testing. --- dbms/programs/server/config.xml | 4 ++-- dbms/tests/server-test.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 5deb0c4e108..108e64e3387 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -196,8 +196,8 @@ - 127.0.0.255 - 9000 + localhost + 1 diff --git a/dbms/tests/server-test.xml b/dbms/tests/server-test.xml index da68c5bb25d..c20d34cce3f 100644 --- a/dbms/tests/server-test.xml +++ b/dbms/tests/server-test.xml @@ -62,8 +62,8 @@ - 127.0.0.255 - 59000 + localhost + 1 From b7566a8d4db95cb304fff8315bb129e6e9b2811f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 17 Dec 2018 23:07:31 +0300 Subject: [PATCH 044/181] Add manual implementation of trim base case (without character override) --- .../src/Functions/registerFunctionsString.cpp | 2 + dbms/src/Functions/trim.cpp | 138 ++++++++++++++++++ dbms/src/Parsers/ExpressionElementParsers.cpp | 15 +- .../performance/trim/trim_whitespace.xml | 34 +++++ dbms/tests/performance/trim/whitespaces.sql | 17 +++ 5 files changed, 200 insertions(+), 6 deletions(-) create mode 100644 dbms/src/Functions/trim.cpp create mode 100644 dbms/tests/performance/trim/trim_whitespace.xml create mode 100644 dbms/tests/performance/trim/whitespaces.sql diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp index d838ac9ff31..15d37d939b0 100644 --- a/dbms/src/Functions/registerFunctionsString.cpp +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -21,6 +21,7 @@ void registerFunctionSubstringUTF8(FunctionFactory &); void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); void registerFunctionStartsWith(FunctionFactory &); void registerFunctionEndsWith(FunctionFactory &); +void registerFunctionTrim(FunctionFactory &); void registerFunctionRegexpQuoteMeta(FunctionFactory &); #if USE_BASE64 @@ -47,6 +48,7 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionAppendTrailingCharIfAbsent(factory); registerFunctionStartsWith(factory); registerFunctionEndsWith(factory); + registerFunctionTrim(factory); registerFunctionRegexpQuoteMeta(factory); #if USE_BASE64 registerFunctionBase64Encode(factory); diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp new file mode 100644 index 00000000000..f43b19e8268 --- /dev/null +++ b/dbms/src/Functions/trim.cpp @@ -0,0 +1,138 @@ +#include +#include +#include + +#if __SSE4_2__ +#include +#endif + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct TrimModeLeft { + static constexpr auto name = "trimLeft"; + static constexpr bool trim_left = true; + static constexpr bool trim_right = false; +}; + +struct TrimModeRight { + static constexpr auto name = "trimRight"; + static constexpr bool trim_left = false; + static constexpr bool trim_right = true; +}; + +struct TrimModeBoth { + static constexpr auto name = "trimBoth"; + static constexpr bool trim_left = true; + static constexpr bool trim_right = true; +}; + +template +class FunctionTrimImpl +{ +public: + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + size_t size = offsets.size(); + res_offsets.resize(size); + res_data.reserve(data.size()); + + size_t prev_offset = 0; + size_t res_offset = 0; + + const UInt8 * start; + size_t length; + + for (size_t i = 0; i < size; ++i) + { + execute(reinterpret_cast(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length); + + res_data.resize(res_data.size() + length + 1); + std::memcpy(&res_data[res_offset], start, length); + res_offset += length + 1; + res_data[res_offset - 1] = 0; + + res_offsets[i] = res_offset; + prev_offset = offsets[i]; + } + } + + static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception("Functions trimLeft, trimRight and trimBoth cannot work with FixedString argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + +private: + static void execute(const UInt8 * data, size_t size, const UInt8 *& res_data, size_t & res_size) + { + size_t chars_to_trim_left = 0; + size_t chars_to_trim_right = 0; + char whitespace = ' '; +#if __SSE4_2__ + const auto bytes_sse = sizeof(__m128i); + const auto size_sse = size - (size % bytes_sse); + const static std::string whitespace_string(bytes_sse, whitespace); + const auto whitespace_mask = _mm_loadu_si128(reinterpret_cast(whitespace_string.data())); + constexpr auto base_sse_mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY; + constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT; + constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; + auto mask = bytes_sse; +#endif + + if constexpr (mode::trim_left) { +#if __SSE4_2__ + /// skip whitespace from left in blocks of up to 16 characters + while (mask == bytes_sse && chars_to_trim_left < size_sse) + { + const auto chars = _mm_loadu_si128(reinterpret_cast(data + chars_to_trim_left)); + mask = _mm_cmpistri(whitespace_mask, chars, left_sse_mode); + chars_to_trim_left += mask; + } +#endif + /// skip remaining whitespace from left, character by character + while (chars_to_trim_left < size && data[chars_to_trim_left] == whitespace) + ++chars_to_trim_left; + } + + if constexpr (mode::trim_right) { + const auto trim_right_size = size - chars_to_trim_left; +#if __SSE4_2__ + /// try to skip whitespace from right in blocks of up to 16 characters + const auto trim_right_size_sse = trim_right_size - (trim_right_size % bytes_sse); + while (mask == bytes_sse && chars_to_trim_right < trim_right_size_sse) + { + const auto chars = _mm_loadu_si128(reinterpret_cast(data + size - chars_to_trim_right - bytes_sse)); + mask = _mm_cmpistri(whitespace_mask, chars, right_sse_mode); + chars_to_trim_right += mask; + } +#endif + /// skip remaining whitespace from right, character by character + while (chars_to_trim_right < trim_right_size && data[size - chars_to_trim_right - 1] == whitespace) + ++chars_to_trim_right; + } + + res_data = data + chars_to_trim_left; + res_size = size - chars_to_trim_left - chars_to_trim_right; + } +}; + +using FunctionTrimLeft = FunctionStringToString, TrimModeLeft>; +using FunctionTrimRight = FunctionStringToString, TrimModeRight>; +using FunctionTrimBoth = FunctionStringToString, TrimModeBoth>; + +void registerFunctionTrim(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); +} +} diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 808be36b1f2..ebbeb6bceac 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -392,7 +392,7 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect { /// Handles all possible TRIM/LTRIM/RTRIM call variants - std::string func_name = "replaceRegexpOne"; + std::string func_name; bool trim_left = false; bool trim_right = false; bool char_override = false; @@ -507,6 +507,7 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect std::make_shared("]*$") }; } + func_name = "replaceRegexpOne"; } pattern_func_node->name = "concat"; @@ -519,25 +520,27 @@ bool ParserTrimExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expect { if (trim_left && trim_right) { - pattern_node = std::make_shared("^ *| *$"); - func_name = "replaceRegexpAll"; + func_name = "trimBoth"; } else { if (trim_left) { - pattern_node = std::make_shared("^ *"); + func_name = "trimLeft"; } else { /// trim_right == false not possible - pattern_node = std::make_shared(" *$"); + func_name = "trimRight"; } } } auto expr_list_args = std::make_shared(); - expr_list_args->children = {expr_node, pattern_node, std::make_shared("")}; + if (char_override) + expr_list_args->children = {expr_node, pattern_node, std::make_shared("")}; + else + expr_list_args->children = {expr_node}; auto func_node = std::make_shared(); func_node->name = func_name; diff --git a/dbms/tests/performance/trim/trim_whitespace.xml b/dbms/tests/performance/trim/trim_whitespace.xml new file mode 100644 index 00000000000..1d76e2be6c9 --- /dev/null +++ b/dbms/tests/performance/trim/trim_whitespace.xml @@ -0,0 +1,34 @@ + + trim_whitespaces + loop + + + whitespaces + + + + + 30000 + + + + + + + + + + func + + trimLeft(value) + trimRight(value) + trimBoth(value) + replaceRegexpOne(value, '^ *', '') + replaceRegexpOne(value, ' *$', '') + replaceRegexpAll(value, '^ *| *$', '') + + + + + SELECT count() FROM whitespaces WHERE NOT ignore({func}) + diff --git a/dbms/tests/performance/trim/whitespaces.sql b/dbms/tests/performance/trim/whitespaces.sql new file mode 100644 index 00000000000..653bd2e7a5a --- /dev/null +++ b/dbms/tests/performance/trim/whitespaces.sql @@ -0,0 +1,17 @@ +CREATE TABLE whitespaces +( + value String +) +ENGINE = MergeTree() +PARTITION BY tuple() +ORDER BY tuple() + +INSERT INTO whitespaces SELECT value +FROM +( + SELECT + arrayStringConcat(groupArray(' ')) AS spaces, + concat(spaces, toString(any(number)), spaces) AS value + FROM numbers(100000000) + GROUP BY pow(number, intHash32(number) % 4) % 12345678 +) -- repeat something like this multiple times and/or just copy whitespaces table into itself From 93d4303922ccead6e5251bb1fe212e47f67d4777 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 17 Dec 2018 23:20:25 +0300 Subject: [PATCH 045/181] minor trim test improvements --- dbms/tests/performance/trim/trim_whitespace.xml | 1 + .../0_stateless/00765_sql_compatibility_aliases.reference | 1 + .../queries/0_stateless/00765_sql_compatibility_aliases.sql | 1 + 3 files changed, 3 insertions(+) diff --git a/dbms/tests/performance/trim/trim_whitespace.xml b/dbms/tests/performance/trim/trim_whitespace.xml index 1d76e2be6c9..d7fc5d967a6 100644 --- a/dbms/tests/performance/trim/trim_whitespace.xml +++ b/dbms/tests/performance/trim/trim_whitespace.xml @@ -20,6 +20,7 @@ func + value trimLeft(value) trimRight(value) trimBoth(value) diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index 691d92957a4..297a6e93d4f 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -14,6 +14,7 @@ fo foo r bar + foo foo xxfoo diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index f01560587a8..f54a0466558 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -16,6 +16,7 @@ select left('foo', 2); select LEFT('foo', 123); select RIGHT('bar', 1); select right('bar', 123); +select ltrim('') || rtrim('') || trim(''); select ltrim(' foo'); select RTRIM(' foo '); select trim(TRAILING 'x' FROM 'xxfooxx'); From 5caab32340e86d13edc107fc64b10d3beac63815 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 11:30:25 +0300 Subject: [PATCH 046/181] simplify whitespace_mask creation --- dbms/src/Functions/trim.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index f43b19e8268..deee24fde5d 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -80,8 +80,7 @@ private: #if __SSE4_2__ const auto bytes_sse = sizeof(__m128i); const auto size_sse = size - (size % bytes_sse); - const static std::string whitespace_string(bytes_sse, whitespace); - const auto whitespace_mask = _mm_loadu_si128(reinterpret_cast(whitespace_string.data())); + const auto whitespace_mask = _mm_set1_epi8(whitespace); constexpr auto base_sse_mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY; constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT; constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; From 63014df313b0cad0b2eabf9532e45a36f010796a Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 18 Dec 2018 14:36:55 +0300 Subject: [PATCH 047/181] Fix macos build --- cmake/find_hdfs3.cmake | 9 ++++++--- cmake/find_libgsasl.cmake | 4 ++-- cmake/find_rdkafka.cmake | 2 +- cmake/find_ssl.cmake | 6 +++--- cmake/find_zlib.cmake | 2 +- dbms/src/Storages/MergeTree/DiskSpaceMonitor.cpp | 2 +- debian/.pbuilderrc | 2 ++ libs/libcommon/cmake/find_jemalloc.cmake | 2 +- 8 files changed, 17 insertions(+), 12 deletions(-) diff --git a/cmake/find_hdfs3.cmake b/cmake/find_hdfs3.cmake index a6fdec20291..a30409b50d5 100644 --- a/cmake/find_hdfs3.cmake +++ b/cmake/find_hdfs3.cmake @@ -15,12 +15,15 @@ if (NOT USE_INTERNAL_HDFS3_LIBRARY) endif () if (HDFS3_LIBRARY AND HDFS3_INCLUDE_DIR) -else () + set(USE_HDFS 1) +elseif (LIBGSASL_LIBRARY) set(HDFS3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include") set(HDFS3_LIBRARY hdfs3) + set(USE_HDFS 1) +else() + set(USE_INTERNAL_HDFS3_LIBRARY 0) endif() -set (USE_HDFS 1) endif() -message (STATUS "Using hdfs3: ${HDFS3_INCLUDE_DIR} : ${HDFS3_LIBRARY}") +message (STATUS "Using hdfs3=${USE_HDFS}: ${HDFS3_INCLUDE_DIR} : ${HDFS3_LIBRARY}") diff --git a/cmake/find_libgsasl.cmake b/cmake/find_libgsasl.cmake index b686bb1df7f..0c88bd2c24a 100644 --- a/cmake/find_libgsasl.cmake +++ b/cmake/find_libgsasl.cmake @@ -1,4 +1,4 @@ -if (NOT APPLE) +if (NOT APPLE AND NOT ARCH_32) option (USE_INTERNAL_LIBGSASL_LIBRARY "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED}) endif () @@ -13,7 +13,7 @@ if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) endif () if (LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) -else () +elseif (NOT APPLE AND NOT ARCH_32) set (LIBGSASL_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include) set (USE_INTERNAL_LIBGSASL_LIBRARY 1) set (LIBGSASL_LIBRARY libgsasl) diff --git a/cmake/find_rdkafka.cmake b/cmake/find_rdkafka.cmake index 1d2674ea1a3..9ba48cadfcd 100644 --- a/cmake/find_rdkafka.cmake +++ b/cmake/find_rdkafka.cmake @@ -1,4 +1,4 @@ -if (NOT ARCH_ARM) +if (NOT ARCH_ARM AND NOT ARCH_32) option (ENABLE_RDKAFKA "Enable kafka" ON) endif () diff --git a/cmake/find_ssl.cmake b/cmake/find_ssl.cmake index 6d630c7198c..d159bcfba23 100644 --- a/cmake/find_ssl.cmake +++ b/cmake/find_ssl.cmake @@ -1,6 +1,6 @@ -#if (OS_LINUX) -option (USE_INTERNAL_SSL_LIBRARY "Set to FALSE to use system *ssl library instead of bundled" ${NOT_UNBUNDLED}) -#endif () +if(NOT ARCH_32) + option(USE_INTERNAL_SSL_LIBRARY "Set to FALSE to use system *ssl library instead of bundled" ${NOT_UNBUNDLED}) +endif() set (OPENSSL_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES}) diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index f6b2d268291..90fe674ab36 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -1,4 +1,4 @@ -if (NOT OS_FREEBSD) +if (NOT OS_FREEBSD AND NOT ARCH_32) option (USE_INTERNAL_ZLIB_LIBRARY "Set to FALSE to use system zlib library instead of bundled" ${NOT_UNBUNDLED}) endif () diff --git a/dbms/src/Storages/MergeTree/DiskSpaceMonitor.cpp b/dbms/src/Storages/MergeTree/DiskSpaceMonitor.cpp index e5bb80df138..507ac8b7459 100644 --- a/dbms/src/Storages/MergeTree/DiskSpaceMonitor.cpp +++ b/dbms/src/Storages/MergeTree/DiskSpaceMonitor.cpp @@ -4,7 +4,7 @@ namespace DB { UInt64 DiskSpaceMonitor::reserved_bytes; -size_t DiskSpaceMonitor::reservation_count; +UInt64 DiskSpaceMonitor::reservation_count; std::mutex DiskSpaceMonitor::mutex; } diff --git a/debian/.pbuilderrc b/debian/.pbuilderrc index 4e5722fe9e5..12e275836c2 100644 --- a/debian/.pbuilderrc +++ b/debian/.pbuilderrc @@ -192,6 +192,8 @@ EXTRAPACKAGES+=" psmisc " [[ $CCACHE_PREFIX == 'distcc' ]] && EXTRAPACKAGES+=" $CCACHE_PREFIX " && USENETWORK=yes && export DISTCC_DIR=/var/cache/pbuilder/distcc +[[ $ARCH == 'i386' ]] && EXTRAPACKAGES+=" libssl-dev " + export DEB_BUILD_OPTIONS=parallel=`nproc` # Floating bug with permissions: diff --git a/libs/libcommon/cmake/find_jemalloc.cmake b/libs/libcommon/cmake/find_jemalloc.cmake index f3fa138e5cc..d9bc37f9d6c 100644 --- a/libs/libcommon/cmake/find_jemalloc.cmake +++ b/libs/libcommon/cmake/find_jemalloc.cmake @@ -1,4 +1,4 @@ -if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM) +if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32) set(ENABLE_JEMALLOC_DEFAULT 1) else () set(ENABLE_JEMALLOC_DEFAULT 0) From 362948e891cc71f8b89d15b007998d0f8d0d588e Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 16:16:48 +0300 Subject: [PATCH 048/181] Introduce quarter interval kind as 3 months (in preparation for #3705) --- dbms/src/DataTypes/DataTypeInterval.cpp | 1 + dbms/src/DataTypes/DataTypeInterval.h | 2 ++ .../FunctionDateOrDateTimeAddInterval.h | 16 ++++++++++++++++ dbms/src/Functions/FunctionsConversion.cpp | 1 + dbms/src/Functions/FunctionsConversion.h | 1 + dbms/src/Functions/addQuarters.cpp | 18 ++++++++++++++++++ .../Functions/registerFunctionsDateTime.cpp | 5 ++++- dbms/src/Functions/subtractQuarters.cpp | 18 ++++++++++++++++++ dbms/src/Parsers/ExpressionListParsers.cpp | 2 ++ .../00514_interval_operators.reference | 1 + .../0_stateless/00514_interval_operators.sql | 1 + libs/libcommon/include/common/DateLUTImpl.h | 10 ++++++++++ 12 files changed, 75 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Functions/addQuarters.cpp create mode 100644 dbms/src/Functions/subtractQuarters.cpp diff --git a/dbms/src/DataTypes/DataTypeInterval.cpp b/dbms/src/DataTypes/DataTypeInterval.cpp index ab2993b884a..c7ee3ede334 100644 --- a/dbms/src/DataTypes/DataTypeInterval.cpp +++ b/dbms/src/DataTypes/DataTypeInterval.cpp @@ -19,6 +19,7 @@ void registerDataTypeInterval(DataTypeFactory & factory) factory.registerSimpleDataType("IntervalDay", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Day)); }); factory.registerSimpleDataType("IntervalWeek", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Week)); }); factory.registerSimpleDataType("IntervalMonth", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Month)); }); + factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Quarter)); }); factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Year)); }); } diff --git a/dbms/src/DataTypes/DataTypeInterval.h b/dbms/src/DataTypes/DataTypeInterval.h index afbcf2d6a45..6f4f08c16c0 100644 --- a/dbms/src/DataTypes/DataTypeInterval.h +++ b/dbms/src/DataTypes/DataTypeInterval.h @@ -25,6 +25,7 @@ public: Day, Week, Month, + Quarter, Year }; @@ -46,6 +47,7 @@ public: case Day: return "Day"; case Week: return "Week"; case Month: return "Month"; + case Quarter: return "Quarter"; case Year: return "Year"; default: __builtin_unreachable(); } diff --git a/dbms/src/Functions/FunctionDateOrDateTimeAddInterval.h b/dbms/src/Functions/FunctionDateOrDateTimeAddInterval.h index c4b7639908f..9b27282ec19 100644 --- a/dbms/src/Functions/FunctionDateOrDateTimeAddInterval.h +++ b/dbms/src/Functions/FunctionDateOrDateTimeAddInterval.h @@ -113,6 +113,21 @@ struct AddMonthsImpl } }; +struct AddQuartersImpl +{ + static constexpr auto name = "addQuarters"; + + static inline UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone) + { + return time_zone.addQuarters(t, delta); + } + + static inline UInt16 execute(UInt16 d, Int64 delta, const DateLUTImpl & time_zone) + { + return time_zone.addQuarters(DayNum(d), delta); + } +}; + struct AddYearsImpl { static constexpr auto name = "addYears"; @@ -149,6 +164,7 @@ struct SubtractHoursImpl : SubtractIntervalImpl { static constexpr struct SubtractDaysImpl : SubtractIntervalImpl { static constexpr auto name = "subtractDays"; }; struct SubtractWeeksImpl : SubtractIntervalImpl { static constexpr auto name = "subtractWeeks"; }; struct SubtractMonthsImpl : SubtractIntervalImpl { static constexpr auto name = "subtractMonths"; }; +struct SubtractQuartersImpl : SubtractIntervalImpl { static constexpr auto name = "subtractQuarters"; }; struct SubtractYearsImpl : SubtractIntervalImpl { static constexpr auto name = "subtractYears"; }; diff --git a/dbms/src/Functions/FunctionsConversion.cpp b/dbms/src/Functions/FunctionsConversion.cpp index fdfc153f594..a83a756010c 100644 --- a/dbms/src/Functions/FunctionsConversion.cpp +++ b/dbms/src/Functions/FunctionsConversion.cpp @@ -89,6 +89,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction>(); + factory.registerFunction>(); factory.registerFunction>(); } diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 1428fec4f48..6b42bec10df 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -738,6 +738,7 @@ DEFINE_NAME_TO_INTERVAL(Hour) DEFINE_NAME_TO_INTERVAL(Day) DEFINE_NAME_TO_INTERVAL(Week) DEFINE_NAME_TO_INTERVAL(Month) +DEFINE_NAME_TO_INTERVAL(Quarter) DEFINE_NAME_TO_INTERVAL(Year) #undef DEFINE_NAME_TO_INTERVAL diff --git a/dbms/src/Functions/addQuarters.cpp b/dbms/src/Functions/addQuarters.cpp new file mode 100644 index 00000000000..c37fb5561c8 --- /dev/null +++ b/dbms/src/Functions/addQuarters.cpp @@ -0,0 +1,18 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionAddQuarters = FunctionDateOrDateTimeAddInterval; + +void registerFunctionAddQuarters(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + + diff --git a/dbms/src/Functions/registerFunctionsDateTime.cpp b/dbms/src/Functions/registerFunctionsDateTime.cpp index 3e7f2a6affd..fe2734a7b75 100644 --- a/dbms/src/Functions/registerFunctionsDateTime.cpp +++ b/dbms/src/Functions/registerFunctionsDateTime.cpp @@ -47,6 +47,7 @@ void registerFunctionAddHours(FunctionFactory &); void registerFunctionAddDays(FunctionFactory &); void registerFunctionAddWeeks(FunctionFactory &); void registerFunctionAddMonths(FunctionFactory &); +void registerFunctionAddQuarters(FunctionFactory &); void registerFunctionAddYears(FunctionFactory &); void registerFunctionSubtractSeconds(FunctionFactory &); void registerFunctionSubtractMinutes(FunctionFactory &); @@ -54,6 +55,7 @@ void registerFunctionSubtractHours(FunctionFactory &); void registerFunctionSubtractDays(FunctionFactory &); void registerFunctionSubtractWeeks(FunctionFactory &); void registerFunctionSubtractMonths(FunctionFactory &); +void registerFunctionSubtractQuarters(FunctionFactory &); void registerFunctionSubtractYears(FunctionFactory &); void registerFunctionDateDiff(FunctionFactory &); void registerFunctionToTimeZone(FunctionFactory &); @@ -106,13 +108,14 @@ void registerFunctionsDateTime(FunctionFactory & factory) registerFunctionAddDays(factory); registerFunctionAddWeeks(factory); registerFunctionAddMonths(factory); + registerFunctionAddQuarters(factory); registerFunctionAddYears(factory); registerFunctionSubtractSeconds(factory); registerFunctionSubtractMinutes(factory); registerFunctionSubtractHours(factory); registerFunctionSubtractDays(factory); registerFunctionSubtractWeeks(factory); - registerFunctionSubtractMonths(factory); + registerFunctionSubtractQuarters(factory); registerFunctionSubtractYears(factory); registerFunctionDateDiff(factory); registerFunctionToTimeZone(factory); diff --git a/dbms/src/Functions/subtractQuarters.cpp b/dbms/src/Functions/subtractQuarters.cpp new file mode 100644 index 00000000000..6c066ed17a1 --- /dev/null +++ b/dbms/src/Functions/subtractQuarters.cpp @@ -0,0 +1,18 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionSubtractQuarters = FunctionDateOrDateTimeAddInterval; + +void registerFunctionSubtractQuarters(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + + diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index ef75267cffe..9365606f5dd 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -621,6 +621,8 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec function_name = "toIntervalWeek"; else if (ParserKeyword("MONTH").ignore(pos, expected)) function_name = "toIntervalMonth"; + else if (ParserKeyword("QUARTER").ignore(pos, expected)) + function_name = "toIntervalQuarter"; else if (ParserKeyword("YEAR").ignore(pos, expected)) function_name = "toIntervalYear"; else diff --git a/dbms/tests/queries/0_stateless/00514_interval_operators.reference b/dbms/tests/queries/0_stateless/00514_interval_operators.reference index 8af8f56eb87..43238eecb3d 100644 --- a/dbms/tests/queries/0_stateless/00514_interval_operators.reference +++ b/dbms/tests/queries/0_stateless/00514_interval_operators.reference @@ -36,3 +36,4 @@ 2029-02-28 01:02:03 2017-03-29 01:02:03 2030-02-28 01:02:03 2017-04-29 01:02:03 2031-02-28 01:02:03 2017-05-29 01:02:03 +2015-11-29 01:02:03 diff --git a/dbms/tests/queries/0_stateless/00514_interval_operators.sql b/dbms/tests/queries/0_stateless/00514_interval_operators.sql index 9dc2f67322b..a4b6c983abf 100644 --- a/dbms/tests/queries/0_stateless/00514_interval_operators.sql +++ b/dbms/tests/queries/0_stateless/00514_interval_operators.sql @@ -2,3 +2,4 @@ SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 DAY + INTERVAL 1 MONTH - I SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 HOUR + INTERVAL 1000 MINUTE + INTERVAL 10 SECOND; SELECT toDateTime('2017-10-30 08:18:19') + INTERVAL 1 DAY + INTERVAL number MONTH FROM system.numbers LIMIT 20; SELECT toDateTime('2016-02-29 01:02:03') + INTERVAL number YEAR, toDateTime('2016-02-29 01:02:03') + INTERVAL number MONTH FROM system.numbers LIMIT 16; +SELECT toDateTime('2016-02-29 01:02:03') - INTERVAL 1 QUARTER; diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 56d9cc04dd1..55a94f3733a 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -584,6 +584,16 @@ public: } } + inline time_t addQuarters(time_t t, Int64 delta) const + { + return addMonths(t, delta * 3); + } + + inline DayNum addQuarters(DayNum d, Int64 delta) const + { + return addMonths(d, delta * 3); + } + /// Saturation can occur if 29 Feb is mapped to non-leap year. inline time_t addYears(time_t t, Int64 delta) const { From d7667ffe95ace32cd1cd95bb352251ceca02ce49 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 16:18:30 +0300 Subject: [PATCH 049/181] fix gcc build issue --- dbms/src/Functions/trim.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index deee24fde5d..6b09c643077 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -82,14 +82,13 @@ private: const auto size_sse = size - (size % bytes_sse); const auto whitespace_mask = _mm_set1_epi8(whitespace); constexpr auto base_sse_mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_EACH | _SIDD_NEGATIVE_POLARITY; - constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT; - constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; auto mask = bytes_sse; #endif if constexpr (mode::trim_left) { #if __SSE4_2__ /// skip whitespace from left in blocks of up to 16 characters + constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT; while (mask == bytes_sse && chars_to_trim_left < size_sse) { const auto chars = _mm_loadu_si128(reinterpret_cast(data + chars_to_trim_left)); @@ -103,6 +102,7 @@ private: } if constexpr (mode::trim_right) { + constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; const auto trim_right_size = size - chars_to_trim_left; #if __SSE4_2__ /// try to skip whitespace from right in blocks of up to 16 characters From d59b7379c94f68fe61e8288ccc3c5e5e1224e45a Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 16:19:23 +0300 Subject: [PATCH 050/181] fix style issues --- dbms/src/Functions/trim.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index 6b09c643077..f6f9d22496a 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -14,19 +14,22 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -struct TrimModeLeft { +struct TrimModeLeft +{ static constexpr auto name = "trimLeft"; static constexpr bool trim_left = true; static constexpr bool trim_right = false; }; -struct TrimModeRight { +struct TrimModeRight +{ static constexpr auto name = "trimRight"; static constexpr bool trim_left = false; static constexpr bool trim_right = true; }; -struct TrimModeBoth { +struct TrimModeBoth +{ static constexpr auto name = "trimBoth"; static constexpr bool trim_left = true; static constexpr bool trim_right = true; @@ -85,7 +88,8 @@ private: auto mask = bytes_sse; #endif - if constexpr (mode::trim_left) { + if constexpr (mode::trim_left) + { #if __SSE4_2__ /// skip whitespace from left in blocks of up to 16 characters constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT; @@ -101,7 +105,8 @@ private: ++chars_to_trim_left; } - if constexpr (mode::trim_right) { + if constexpr (mode::trim_right) + { constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; const auto trim_right_size = size - chars_to_trim_left; #if __SSE4_2__ From e73bef119c34c778ac675cab7d317f88cd619442 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 17:31:03 +0300 Subject: [PATCH 051/181] restore accidentally removed function registration --- dbms/src/Functions/registerFunctionsDateTime.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/registerFunctionsDateTime.cpp b/dbms/src/Functions/registerFunctionsDateTime.cpp index fe2734a7b75..5751fc800a9 100644 --- a/dbms/src/Functions/registerFunctionsDateTime.cpp +++ b/dbms/src/Functions/registerFunctionsDateTime.cpp @@ -115,6 +115,7 @@ void registerFunctionsDateTime(FunctionFactory & factory) registerFunctionSubtractHours(factory); registerFunctionSubtractDays(factory); registerFunctionSubtractWeeks(factory); + registerFunctionSubtractMonths(factory); registerFunctionSubtractQuarters(factory); registerFunctionSubtractYears(factory); registerFunctionDateDiff(factory); From 9ca9f600ad9c21ddcf561565fa2963f47d80a619 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 18 Dec 2018 17:31:20 +0300 Subject: [PATCH 052/181] Fix macos build --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 4 ++-- dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index e09ac52a701..0d6cdd3f557 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -45,12 +45,12 @@ public: /** Get maximum total size of parts to do merge, at current moment of time. * It depends on number of free threads in background_pool and amount of free space in disk. */ - size_t getMaxSourcePartsSize(); + UInt64 getMaxSourcePartsSize(); /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ - size_t getMaxSourcePartsSize(size_t pool_size, size_t pool_used); + UInt64 getMaxSourcePartsSize(size_t pool_size, size_t pool_used); /** Selects which parts to merge. Uses a lot of heuristics. * diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 41a8f8ee224..2499b3239e7 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -900,7 +900,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( * But if all threads are free (maximal size of merge is allowed) then execute any merge, * (because it may be ordered by OPTIMIZE or early with differrent settings). */ - size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); if (max_source_parts_size != data.settings.max_bytes_to_merge_at_max_space_in_pool && sum_parts_size_in_bytes > max_source_parts_size) { diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d034fe67bd8..f4f69e3ac87 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -418,7 +418,7 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { - size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); if (max_source_parts_size > 0) selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, out_disable_reason); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index e4bfb4c90d5..10981823b66 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2160,7 +2160,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() } else { - size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize( + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSize( data.settings.max_replicated_merges_in_queue, merges_and_mutations_queued); if (max_source_parts_size > 0) From 5abfcfcd4814bb4133a19749153668f614c0b9e1 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 17:40:30 +0300 Subject: [PATCH 053/181] Support proper syntax for dateDiff --- dbms/src/Parsers/ExpressionElementParsers.cpp | 65 +++++++++++++++++++ dbms/src/Parsers/ExpressionElementParsers.h | 6 ++ .../00765_sql_compatibility_aliases.reference | 1 + .../00765_sql_compatibility_aliases.sql | 1 + 4 files changed, 73 insertions(+) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index ebbeb6bceac..d189b178de9 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -698,6 +698,70 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } +bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + const char * interval_name = nullptr; + ASTPtr left_node; + ASTPtr right_node; + + if (!(ParserKeyword("DATEDIFF").ignore(pos, expected) || ParserKeyword("DATE_DIFF").ignore(pos, expected))) + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + + if (ParserKeyword("SECOND").ignore(pos, expected)) + interval_name = "second"; + else if (ParserKeyword("MINUTE").ignore(pos, expected)) + interval_name = "minute"; + else if (ParserKeyword("HOUR").ignore(pos, expected)) + interval_name = "hour"; + else if (ParserKeyword("DAY").ignore(pos, expected)) + interval_name = "day"; + else if (ParserKeyword("WEEK").ignore(pos, expected)) + interval_name = "week"; + else if (ParserKeyword("MONTH").ignore(pos, expected)) + interval_name = "month"; + else if (ParserKeyword("QUARTER").ignore(pos, expected)) + interval_name = "quarter"; + else if (ParserKeyword("YEAR").ignore(pos, expected)) + interval_name = "year"; + else + return false; + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, left_node, expected)) + return false; + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, right_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {std::make_shared(interval_name), left_node, right_node}; + + auto func_node = std::make_shared(); + func_node->name = "dateDiff"; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + + return true; +} + bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { @@ -999,6 +1063,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserLiteral().parse(pos, node, expected) || ParserCastExpression().parse(pos, node, expected) || ParserExtractExpression().parse(pos, node, expected) + || ParserDateDiffExpression().parse(pos, node, expected) || ParserSubstringExpression().parse(pos, node, expected) || ParserTrimExpression().parse(pos, node, expected) || ParserLeftExpression().parse(pos, node, expected) diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index 98aec4fa46c..c35a6613155 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -131,6 +131,12 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserDateDiffExpression : public IParserBase +{ +protected: + const char * getName() const override { return "DATE_DIFF expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; /** NULL literal. */ diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index 297a6e93d4f..be273b59160 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -21,3 +21,4 @@ xxfoo fooabba fooabbafoo foo* +-11 diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index f54a0466558..79812497656 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -23,3 +23,4 @@ select trim(TRAILING 'x' FROM 'xxfooxx'); select Trim(LEADING 'ab' FROM 'abbafooabba'); select TRIM(both 'ab' FROM 'abbafooabbafooabba'); select trim(LEADING '*[]{}|\\' FROM '\\|[[[}}}*foo*'); +select DATE_DIFF(MONTH, toDate('2018-12-18'), toDate('2018-01-01')); From a378b919af586164b3571eca5e8f0a8a6cb64da2 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 18:28:13 +0300 Subject: [PATCH 054/181] Add a bunch of aliases to DATE_DIFF interval names --- dbms/src/Parsers/ExpressionElementParsers.cpp | 17 ++++++++--------- .../00765_sql_compatibility_aliases.reference | 1 + .../00765_sql_compatibility_aliases.sql | 1 + 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index d189b178de9..17587000a2f 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -711,22 +711,21 @@ bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return false; ++pos; - - if (ParserKeyword("SECOND").ignore(pos, expected)) + if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) interval_name = "second"; - else if (ParserKeyword("MINUTE").ignore(pos, expected)) + else if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) interval_name = "minute"; - else if (ParserKeyword("HOUR").ignore(pos, expected)) + else if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("HH").ignore(pos, expected)) interval_name = "hour"; - else if (ParserKeyword("DAY").ignore(pos, expected)) + else if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) interval_name = "day"; - else if (ParserKeyword("WEEK").ignore(pos, expected)) + else if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) interval_name = "week"; - else if (ParserKeyword("MONTH").ignore(pos, expected)) + else if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) interval_name = "month"; - else if (ParserKeyword("QUARTER").ignore(pos, expected)) + else if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) interval_name = "quarter"; - else if (ParserKeyword("YEAR").ignore(pos, expected)) + else if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) interval_name = "year"; else return false; diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index be273b59160..4e05015901b 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -22,3 +22,4 @@ fooabba fooabbafoo foo* -11 +-3 diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index 79812497656..6fea2dafe1c 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -24,3 +24,4 @@ select Trim(LEADING 'ab' FROM 'abbafooabba'); select TRIM(both 'ab' FROM 'abbafooabbafooabba'); select trim(LEADING '*[]{}|\\' FROM '\\|[[[}}}*foo*'); select DATE_DIFF(MONTH, toDate('2018-12-18'), toDate('2018-01-01')); +select DATE_DIFF(QQ, toDate('2018-12-18'), toDate('2018-01-01')); From c5cb1573b5456b6506431259df359dad5399d8d6 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 18 Dec 2018 18:47:27 +0300 Subject: [PATCH 055/181] One more kind of aliases for DATE_DIFF --- dbms/src/Parsers/ExpressionElementParsers.cpp | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 17587000a2f..6de29f097d8 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -711,21 +711,36 @@ bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return false; ++pos; - if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) + if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected) + || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) interval_name = "second"; - else if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) + else if ( + ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected) + || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) interval_name = "minute"; - else if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("HH").ignore(pos, expected)) + else if ( + ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected) + || ParserKeyword("HH").ignore(pos, expected)) interval_name = "hour"; - else if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) + else if ( + ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected) + || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) interval_name = "day"; - else if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) + else if ( + ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected) + || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) interval_name = "week"; - else if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) + else if ( + ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected) + || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) interval_name = "month"; - else if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) + else if ( + ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected) + || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) interval_name = "quarter"; - else if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) + else if ( + ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected) + || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) interval_name = "year"; else return false; From b213ee5e5de541babcdad096e84e2ff5cdc2c9c2 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 18 Dec 2018 20:41:03 +0300 Subject: [PATCH 056/181] Fix macos build (#3863) --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 4 ++-- dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index e09ac52a701..0d6cdd3f557 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -45,12 +45,12 @@ public: /** Get maximum total size of parts to do merge, at current moment of time. * It depends on number of free threads in background_pool and amount of free space in disk. */ - size_t getMaxSourcePartsSize(); + UInt64 getMaxSourcePartsSize(); /** For explicitly passed size of pool and number of used tasks. * This method could be used to calculate threshold depending on number of tasks in replication queue. */ - size_t getMaxSourcePartsSize(size_t pool_size, size_t pool_used); + UInt64 getMaxSourcePartsSize(size_t pool_size, size_t pool_used); /** Selects which parts to merge. Uses a lot of heuristics. * diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 41a8f8ee224..2499b3239e7 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -900,7 +900,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( * But if all threads are free (maximal size of merge is allowed) then execute any merge, * (because it may be ordered by OPTIMIZE or early with differrent settings). */ - size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); if (max_source_parts_size != data.settings.max_bytes_to_merge_at_max_space_in_pool && sum_parts_size_in_bytes > max_source_parts_size) { diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d034fe67bd8..f4f69e3ac87 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -418,7 +418,7 @@ bool StorageMergeTree::merge( if (partition_id.empty()) { - size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSize(); if (max_source_parts_size > 0) selected = merger_mutator.selectPartsToMerge(future_part, aggressive, max_source_parts_size, can_merge, out_disable_reason); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index e4bfb4c90d5..10981823b66 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2160,7 +2160,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() } else { - size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize( + UInt64 max_source_parts_size = merger_mutator.getMaxSourcePartsSize( data.settings.max_replicated_merges_in_queue, merges_and_mutations_queued); if (max_source_parts_size > 0) From 9a44ccb664d91ece80a6588038ecf572eed3625f Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Tue, 18 Dec 2018 14:12:31 -0400 Subject: [PATCH 057/181] Update CHANGELOG_RU.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit добавил в обратно несовместимые изменения -- Удалена возможность сравнения типа `Date` с числом --- CHANGELOG_RU.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index f023eee4680..629ddf62f11 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -88,6 +88,10 @@ * Для Docker образа добавлена поддержка инициализации базы данных с помощью файлов в директории `/docker-entrypoint-initdb.d`. * Исправления для сборки под ARM. +### Обратно несовместимые изменения: + +* Удалена возможность сравнения типа `Date` с числом, необходимо вместо `toDate('2018-12-18') = 17883`, использовать явное приведение типов `= toDate(17883)` + ## ClickHouse release 18.14.18, 2018-12-04 From 617082f4447aba0b4c8165bbd6c60623ba56784d Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 18 Dec 2018 21:43:06 +0300 Subject: [PATCH 058/181] ISSUE-3590: Forbid using aggregate functions inside scalar subqueries --- .../Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 12 ++++++++++++ .../queries/0_stateless/00205_scalar_subqueries.sql | 3 +++ 2 files changed, 15 insertions(+) diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 9cea690a39b..7a43d7362e6 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -9,6 +9,8 @@ #include #include +#include + namespace DB { @@ -98,6 +100,11 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr size_t columns = block.columns(); if (columns == 1) { + if (typeid_cast(block.safeGetByPosition(0).type.get())) + { + throw Exception("Scalar subquery can't contain an aggregate function", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + } + auto lit = std::make_unique((*block.safeGetByPosition(0).column)[0]); lit->alias = subquery.alias; lit->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name; @@ -116,6 +123,11 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr exp_list->children.resize(columns); for (size_t i = 0; i < columns; ++i) { + if (typeid_cast(block.safeGetByPosition(i).type.get())) + { + throw Exception("Scalar subquery can't contain an aggregate function", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + } + exp_list->children[i] = addTypeConversion( std::make_unique((*block.safeGetByPosition(i).column)[0]), block.safeGetByPosition(i).type->getName()); diff --git a/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql index f924ff291ea..5ebb524dc53 100644 --- a/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql +++ b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql @@ -1,5 +1,8 @@ +SET send_logs_level = 'none'; + SELECT (SELECT (SELECT (SELECT (SELECT (SELECT count() FROM (SELECT * FROM system.numbers LIMIT 10)))))) = (SELECT 10), ((SELECT 1, 'Hello', [1, 2]).3)[1]; SELECT toUInt64((SELECT 9)) IN (SELECT number FROM system.numbers LIMIT 10); SELECT (SELECT toDate('2015-01-02')) = toDate('2015-01-02'), 'Hello' = (SELECT 'Hello'); SELECT (SELECT toDate('2015-01-02'), 'Hello'); SELECT (SELECT toDate('2015-01-02'), 'Hello') AS x, x, identity((SELECT 1)), identity((SELECT 1) AS y); +SELECT (SELECT uniqState('')); -- { serverError 125 } From d5d1c34017729640fcbe72a2bbafaed2523de15d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 18 Dec 2018 23:24:16 +0300 Subject: [PATCH 059/181] Merging #3811 --- dbms/src/Functions/FunctionsHashing.h | 182 +++++++------------------- 1 file changed, 49 insertions(+), 133 deletions(-) diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 37fd9ae2093..3dd72865337 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -41,6 +41,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int NOT_IMPLEMENTED; } @@ -298,6 +299,51 @@ struct MurmurHash3Impl64 static constexpr bool use_int_hash_for_pods = false; }; +/// http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452 +/// Care should be taken to do all calculation in unsigned integers (to avoid undefined behaviour on overflow) +/// but obtain the same result as it is done in singed integers with two's complement arithmetic. +struct JavaHashImpl +{ + static constexpr auto name = "javaHash"; + using ReturnType = Int32; + + static Int32 apply(const char * data, const size_t size) + { + UInt32 h = 0; + for (size_t i = 0; i < size; ++i) + h = 31 * h + static_cast(static_cast(data[i])); + return static_cast(h); + } + + static Int32 combineHashes(Int32, Int32) + { + throw Exception("Java hash is not combineable for multiple arguments", ErrorCodes::NOT_IMPLEMENTED); + } + + static constexpr bool use_int_hash_for_pods = false; +}; + +/// This is just JavaHash with zeroed out sign bit. +/// This function is used in Hive for versions before 3.0, +/// after 3.0, Hive uses murmur-hash3. +struct HiveHashImpl +{ + static constexpr auto name = "hiveHash"; + using ReturnType = Int32; + + static Int32 apply(const char * data, const size_t size) + { + return static_cast(0x7FFFFFFF & static_cast(JavaHashImpl::apply(data, size))); + } + + static Int32 combineHashes(Int32, Int32) + { + throw Exception("Hive hash is not combineable for multiple arguments", ErrorCodes::NOT_IMPLEMENTED); + } + + static constexpr bool use_int_hash_for_pods = false; +}; + struct MurmurHash3Impl128 { static constexpr auto name = "murmurHash3_128"; @@ -956,139 +1002,6 @@ private: } }; -/* - * http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452 - */ -struct JavaHashImpl -{ - static Int32 apply(const char * data, const size_t size) - { - Int32 h = 0; - for (int i = 0; i < (int)size; ++i) - { - h = 31 * h + data[i]; - } - return h; - } -}; - -/* - * the java string hash implement, - * many system from java world use this string hash function or based it - */ -class FunctionJavaHash : public IFunction -{ -public: - static constexpr auto name = "JavaHash"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override { return name; } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 1; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const auto arg_count = arguments.size(); - if (arg_count != 1) - throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " + - toString(arg_count) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - const auto first_arg = arguments.front().get(); - if (!WhichDataType(first_arg).isString()) - throw Exception{"Illegal type " + first_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto arg_count = arguments.size(); - - if (arg_count == 1) - { - const auto col_untyped = block.getByPosition(arguments.front()).column.get(); - - if (const auto col_from = checkAndGetColumn(col_untyped)) - { - const auto size = col_from->size(); - auto col_to = ColumnInt32::create(size); - - const auto & chars = col_from->getChars(); - const auto & offsets = col_from->getOffsets(); - auto & out = col_to->getData(); - - ColumnString::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - out[i] = JavaHashImpl::apply( - reinterpret_cast(&chars[current_offset]), - offsets[i] - current_offset - 1); - - current_offset = offsets[i]; - } - - block.getByPosition(result).column = std::move(col_to); - } - else - throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + - " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - else - throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR}; - } -}; - -/* - * this hive function works for hive-version < 3.0, - * after 3.0, hive use murmur-hash3 - */ -class FunctionHiveHash : public FunctionJavaHash -{ -public: - static constexpr auto name = "HiveHash"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override - { - const auto arg_count = arguments.size(); - - if (arg_count == 1) - { - const auto col_untyped = block.getByPosition(arguments.front()).column.get(); - - if (const auto col_from = checkAndGetColumn(col_untyped)) - { - const auto size = col_from->size(); - auto col_to = ColumnInt32::create(size); - - const auto & chars = col_from->getChars(); - const auto & offsets = col_from->getOffsets(); - auto & out = col_to->getData(); - - ColumnString::Offset current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - out[i] = JavaHashImpl::apply( - reinterpret_cast(&chars[current_offset]), - offsets[i] - current_offset - 1) & 0x7fffffff; - - current_offset = offsets[i]; - } - - block.getByPosition(result).column = std::move(col_to); - } - else - throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + - " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - } - else - throw Exception{"got into IFunction::execute with unexpected number of arguments", ErrorCodes::LOGICAL_ERROR}; - } -}; struct NameIntHash32 { static constexpr auto name = "intHash32"; }; struct NameIntHash64 { static constexpr auto name = "intHash64"; }; @@ -1111,4 +1024,7 @@ using FunctionMurmurHash2_64 = FunctionAnyHash; using FunctionMurmurHash3_32 = FunctionAnyHash; using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; +using FunctionJavaHash = FunctionAnyHash; +using FunctionHiveHash = FunctionAnyHash; + } From bec30514889dbfce7ae0ced91fe8dd831b53b804 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 00:34:09 +0300 Subject: [PATCH 060/181] Changelog for 18.16.0: added authors and references [#CLICKHOUSE-2] --- CHANGELOG_RU.md | 153 ++++++++++++++++++++++++------------------------ 1 file changed, 77 insertions(+), 76 deletions(-) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 629ddf62f11..ff36f3e7d64 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -2,95 +2,96 @@ ### Новые возможности: -* Вычисление `DEFAULT` выражений для отсутствующих полей при загрузке данных в полуструктурированных форматах (`JSONEachRow`, `TSKV`). -* Для запроса `ALTER TABLE` добавлено действие `MODIFY ORDER BY` для изменения ключа сортировки при одновременном добавлении или удалении столбца таблицы. Это полезно для таблиц семейства `MergeTree`, выполняющих дополнительную работу при слияниях, согласно этому ключу сортировки, как например, `SummingMergeTree`, `AggregatingMergeTree` и т. п. -* Для таблиц семейства `MergeTree` появилась возможность указать различный ключ сортировки (`ORDER BY`) и индекс (`PRIMARY KEY`). Ключ сортировки может быть длиннее, чем индекс. -* Добавлена табличная функция `hdfs` и движок таблиц `HDFS` для импорта и экспорта данных в HDFS. -* Добавлены функции для работы с base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. -* Для агрегатной функции `uniqCombined` появилась возможность настраивать точность работы с помощью параметра (выбирать количество ячеек HyperLogLog). -* Добавлена таблица `system.contributors`, содержащая имена всех, кто делал коммиты в ClickHouse. -* Добавлена возможность не указывать партицию для запроса `ALTER TABLE ... FREEZE` для бэкапа сразу всех партиций. -* Добавлены функции `dictGet`, `dictGetOrDefault` без указания типа возвращаемого значения. Тип определяется автоматически из описания словаря. -* Возможность указания комментария для столбца в описании таблицы и изменения его с помощью `ALTER`. -* Возможность чтения из таблицы типа `Join` в случае простых ключей. -* Возможность указания настроек `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, `join_overflow_mode` при создании таблицы типа `Join`. -* Добавлена функция `joinGet`, позволяющая использовать таблицы типа `Join` как словарь. -* Добавлены столбцы `partition_key`, `sorting_key`, `primary_key`, `sampling_key` в таблицу `system.tables`, позволяющие получить информацию о ключах таблицы. -* Добавлены столбцы `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, `is_in_sampling_key` в таблицу `system.columns`. -* Добавлены столбцы `min_time`, `max_time` в таблицу `system.parts`. Эти столбцы заполняются, если ключ партиционирования является выражением от столбцов типа `DateTime`. +* Вычисление `DEFAULT` выражений для отсутствующих полей при загрузке данных в полуструктурированных форматах (`JSONEachRow`, `TSKV`). #3555 +* Для запроса `ALTER TABLE` добавлено действие `MODIFY ORDER BY` для изменения ключа сортировки при одновременном добавлении или удалении столбца таблицы. Это полезно для таблиц семейства `MergeTree`, выполняющих дополнительную работу при слияниях, согласно этому ключу сортировки, как например, `SummingMergeTree`, `AggregatingMergeTree` и т. п. #3581 #3755 +* Для таблиц семейства `MergeTree` появилась возможность указать различный ключ сортировки (`ORDER BY`) и индекс (`PRIMARY KEY`). Ключ сортировки может быть длиннее, чем индекс. #3581 +* Добавлена табличная функция `hdfs` и движок таблиц `HDFS` для импорта и экспорта данных в HDFS. chenxing-xc #3617 +* Добавлены функции для работы с base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. Alexander Krasheninnikov #3350 +* Для агрегатной функции `uniqCombined` появилась возможность настраивать точность работы с помощью параметра (выбирать количество ячеек HyperLogLog). #3406 +* Добавлена таблица `system.contributors`, содержащая имена всех, кто делал коммиты в ClickHouse. #3452 +* Добавлена возможность не указывать партицию для запроса `ALTER TABLE ... FREEZE` для бэкапа сразу всех партиций. #3514 +* Добавлены функции `dictGet`, `dictGetOrDefault` без указания типа возвращаемого значения. Тип определяется автоматически из описания словаря. Amos Bird #3564 +* Возможность указания комментария для столбца в описании таблицы и изменения его с помощью `ALTER`. #3377 +* Возможность чтения из таблицы типа `Join` в случае простых ключей. Amos Bird #3728 +* Возможность указания настроек `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, `join_overflow_mode` при создании таблицы типа `Join`. Amos Bird #3728 +* Добавлена функция `joinGet`, позволяющая использовать таблицы типа `Join` как словарь. Amos Bird #3728 +* Добавлены столбцы `partition_key`, `sorting_key`, `primary_key`, `sampling_key` в таблицу `system.tables`, позволяющие получить информацию о ключах таблицы. #3609 +* Добавлены столбцы `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, `is_in_sampling_key` в таблицу `system.columns`. #3609 +* Добавлены столбцы `min_time`, `max_time` в таблицу `system.parts`. Эти столбцы заполняются, если ключ партиционирования является выражением от столбцов типа `DateTime`. Emmanuel Donin de Rosière #3800 ### Исправления ошибок: -* Исправления и улучшения производительности для типа данных `LowCardinality`. `GROUP BY` по `LowCardinality(Nullable(...))`. Получение `extremes` значений. Выполнение функций высшего порядка. `LEFT ARRAY JOIN`. Распределённый `GROUP BY`. Функции, возвращающие `Array`. Выполнение `ORDER BY`. Запись в `Distributed` таблицы (nicelulu). Обратная совместимость для запросов `INSERT` от старых клиентов, реализующих `Native` протокол. Поддержка `LowCardinality` для `JOIN`. Производительность при работе в один поток. -* Исправлена работа настройки `select_sequential_consistency`. Ранее, при включенной настройке, после начала записи в новую партицию, мог возвращаться неполный результат. -* Корректное указание базы данных при выполнении DDL запросов `ON CLUSTER`, а также при выполнении `ALTER UPDATE/DELETE`. -* Корректное указание базы данных для подзапросов внутри VIEW. -* Исправлена работа `PREWHERE` с `FINAL` для `VersionedCollapsingMergeTree`. -* Возможность с помощью запроса `KILL QUERY` отмены запросов, которые ещё не начали выполняться из-за ожидания блокировки таблицы. -* Исправлены расчёты с датой и временем в случае, если стрелки часов были переведены назад в полночь (это происходит в Иране, а также было Москве с 1981 по 1983 год). Ранее это приводило к тому, что стрелки часов переводились на сутки раньше, чем нужно, а также приводило к некорректному форматированию даты-с-временем в текстовом виде. -* Исправлена работа некоторых случаев `VIEW` и подзапросов без указания базы данных. -* Исправлен race condition при одновременном чтении из `MATERIALIZED VIEW` и удалением `MATERIALIZED VIEW` из-за отсутствия блокировки внутренней таблицы `MATERIALIZED VIEW`. -* Исправлена ошибка `Lock handler cannot be nullptr.` -* Исправления выполнения запросов при включенной настройке `compile_expressions` (выключена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. +* Исправления и улучшения производительности для типа данных `LowCardinality`. `GROUP BY` по `LowCardinality(Nullable(...))`. Получение `extremes` значений. Выполнение функций высшего порядка. `LEFT ARRAY JOIN`. Распределённый `GROUP BY`. Функции, возвращающие `Array`. Выполнение `ORDER BY`. Запись в `Distributed` таблицы (nicelulu). Обратная совместимость для запросов `INSERT` от старых клиентов, реализующих `Native` протокол. Поддержка `LowCardinality` для `JOIN`. Производительность при работе в один поток. #3823 #3803 #3799 #3769 #3744 #3681 #3651 #3649 #3641 #3632 #3568 #3523 #3518 +* Исправлена работа настройки `select_sequential_consistency`. Ранее, при включенной настройке, после начала записи в новую партицию, мог возвращаться неполный результат. #2863 +* Корректное указание базы данных при выполнении DDL запросов `ON CLUSTER`, а также при выполнении `ALTER UPDATE/DELETE`. #3772 #3460 +* Корректное указание базы данных для подзапросов внутри VIEW. #3521 +* Исправлена работа `PREWHERE` с `FINAL` для `VersionedCollapsingMergeTree`. 7167bfd7b365538f7a91c4307ad77e552ab4e8c1 +* Возможность с помощью запроса `KILL QUERY` отмены запросов, которые ещё не начали выполняться из-за ожидания блокировки таблицы. #3517 +* Исправлены расчёты с датой и временем в случае, если стрелки часов были переведены назад в полночь (это происходит в Иране, а также было Москве с 1981 по 1983 год). Ранее это приводило к тому, что стрелки часов переводились на сутки раньше, чем нужно, а также приводило к некорректному форматированию даты-с-временем в текстовом виде. #3819 +* Исправлена работа некоторых случаев `VIEW` и подзапросов без указания базы данных. Winter Zhang #3521 +* Исправлен race condition при одновременном чтении из `MATERIALIZED VIEW` и удалением `MATERIALIZED VIEW` из-за отсутствия блокировки внутренней таблицы `MATERIALIZED VIEW`. #3404 #3694 +* Исправлена ошибка `Lock handler cannot be nullptr.` #3689 +* Исправления выполнения запросов при включенной настройке `compile_expressions` (выключена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. #3457 * Исправлено падение при указании неконстантного аргумента scale в функциях `toDecimal32/64/128`. -* Исправлена ошибка при попытке вставки в формате `Values` массива с `NULL` элементами в столбец типа `Array` без `Nullable` (в случае `input_format_values_interpret_expressions` = 1). -* Исправлено непрерывное логгирование ошибок в `DDLWorker`, если ZooKeeper недоступен. -* Исправлен тип возвращаемого значения для функций `quantile*` от аргументов типа `Date` и `DateTime`. -* Исправлена работа секции `WITH`, если она задаёт простой алиас без выражений. -* Исправлена обработка запросов с именованными подзапросами и квалифицированными именами столбцов при включенной настройке `enable_optimize_predicate_expression`. -* Исправлена ошибка `Attempt to attach to nullptr thread group` при работе материализованных представлений. -* Исправлено падение при передаче некоторых некорректных аргументов в функцию `arrayReverse`. -* Исправлен buffer overflow в функции `extractURLParameter`. Увеличена производительность. Добавлена корректная обработка строк, содержащих нулевые байты. -* Исправлен buffer overflow в функциях `lowerUTF8`, `upperUTF8`. Удалена возможность выполнения этих функций над аргументами типа `FixedString`. -* Исправлен редкий race condition при удалении таблиц типа `MergeTree`. -* Исправлен race condition при чтении из таблиц типа `Buffer` и одновременном `ALTER` либо `DROP` таблиц назначения. -* Исправлен segfault в случае превышения ограничения `max_temporary_non_const_columns`. +* Исправлена ошибка при попытке вставки в формате `Values` массива с `NULL` элементами в столбец типа `Array` без `Nullable` (в случае `input_format_values_interpret_expressions` = 1). #3487 #3503 +* Исправлено непрерывное логгирование ошибок в `DDLWorker`, если ZooKeeper недоступен. 8f50c620334988b28018213ec0092fe6423847e2 +* Исправлен тип возвращаемого значения для функций `quantile*` от аргументов типа `Date` и `DateTime`. #3580 +* Исправлена работа секции `WITH`, если она задаёт простой алиас без выражений. #3570 +* Исправлена обработка запросов с именованными подзапросами и квалифицированными именами столбцов при включенной настройке `enable_optimize_predicate_expression`. Winter Zhang #3588 +* Исправлена ошибка `Attempt to attach to nullptr thread group` при работе материализованных представлений. Marek Vavruša #3623 +* Исправлено падение при передаче некоторых некорректных аргументов в функцию `arrayReverse`. 73e3a7b662161d6005e7727d8a711b930386b871 +* Исправлен buffer overflow в функции `extractURLParameter`. Увеличена производительность. Добавлена корректная обработка строк, содержащих нулевые байты. 141e9799e49201d84ea8e951d1bed4fb6d3dacb5 +* Исправлен buffer overflow в функциях `lowerUTF8`, `upperUTF8`. Удалена возможность выполнения этих функций над аргументами типа `FixedString`. #3662 +* Исправлен редкий race condition при удалении таблиц типа `MergeTree`. #3680 +* Исправлен race condition при чтении из таблиц типа `Buffer` и одновременном `ALTER` либо `DROP` таблиц назначения. #3719 +* Исправлен segfault в случае превышения ограничения `max_temporary_non_const_columns`. #3788 ### Улучшения: -* Обработанные конфигурационные файлы записываются сервером не в `/etc/clickhouse-server/` директорию, а в директорию `preprocessed_configs` внутри `path`. Это позволяет оставить директорию `/etc/clickhouse-server/` недоступной для записи пользователем `clickhouse`, что повышает безопасность. -* Настройка `min_merge_bytes_to_use_direct_io` выставлена по-умолчанию в 10 GiB. Слияния, образующие крупные куски таблиц семейства MergeTree, будут производиться в режиме `O_DIRECT`, что исключает вымывание кэша. -* Ускорен запуск сервера в случае наличия очень большого количества таблиц. -* Добавлен пул соединений и HTTP `Keep-Alive` для соединения между репликами. -* В случае ошибки синтаксиса запроса, в `HTTP` интерфейсе возвращается код `400 Bad Request` (ранее возвращался код 500). -* Для настройки `join_default_strictness` выбрано значение по-умолчанию `ALL` для совместимости. -* Убрано логгирование в `stderr` из библиотеки `re2` в случае некорректных или сложных регулярных выражений. -* Для движка таблиц `Kafka` TODO -* Функции `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64` теперь работают для произвольного количества аргументов, а также для аргументов-кортежей. -* Функция `arrayReverse` теперь работает с любыми типами массивов. -* Добавлен опциональный параметр - размер слота для функции `timeSlots`. -* Для `FULL` и `RIGHT JOIN` учитывается настройка `max_block_size` для потока неприсоединённых данных из правой таблицы. -* В `clickhouse-benchmark` и `clickhouse-performance-test` добавлен параметр командной строки `--secure` для включения TLS. -* Преобразование типов в случае, если структура таблицы типа `Buffer` не соответствует структуре таблицы назначения. -* Добавлена настройка `tcp_keep_alive_timeout` для включения keep-alive пакетов после неактивности в течение указанного интервала времени. -* Убрано излишнее квотирование значений ключа партиции в таблице `system.parts`, если он состоит из одного столбца. -* Функция деления с остатком работает для типов данных `Date` и `DateTime`. -* Добавлены синонимы функций `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, `MID`. Некоторые имена функций сделаны регистронезависимыми для совместимости со стандартом SQL. Добавлен синтаксический сахар `SUBSTRING(expr FROM start FOR length)` для совместимости с SQL. -* Добавлена возможность фиксации (`mlock`) страниц памяти, соответствующих исполняемому коду `clickhouse-server` для предотвращения вытеснения их из памяти. Возможность выключена по-умолчанию. -* Увеличена производительность чтения с `O_DIRECT` (с включенной опцией `min_bytes_to_use_direct_io`). -* Улучшена производительность работы функции `dictGet...OrDefault` в случае константного аргумента-ключа и неконстантного аргумента-default. -* В функции `firstSignificantSubdomain` добавлена обработка доменов `gov`, `mil`, `edu`. Увеличена производительность работы. +* Обработанные конфигурационные файлы записываются сервером не в `/etc/clickhouse-server/` директорию, а в директорию `preprocessed_configs` внутри `path`. Это позволяет оставить директорию `/etc/clickhouse-server/` недоступной для записи пользователем `clickhouse`, что повышает безопасность. #2443 +* Настройка `min_merge_bytes_to_use_direct_io` выставлена по-умолчанию в 10 GiB. Слияния, образующие крупные куски таблиц семейства MergeTree, будут производиться в режиме `O_DIRECT`, что исключает вымывание кэша. #3504 +* Ускорен запуск сервера в случае наличия очень большого количества таблиц. #3398 +* Добавлен пул соединений и HTTP `Keep-Alive` для соединения между репликами. #3594 +* В случае ошибки синтаксиса запроса, в `HTTP` интерфейсе возвращается код `400 Bad Request` (ранее возвращался код 500). 31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab +* Для настройки `join_default_strictness` выбрано значение по-умолчанию `ALL` для совместимости. 120e2cbe2ff4fbad626c28042d9b28781c805afe +* Убрано логгирование в `stderr` из библиотеки `re2` в случае некорректных или сложных регулярных выражений. #3723 +* Для движка таблиц `Kafka`: проверка наличия подписок перед началом чтения из Kafka; настройка таблицы kafka_max_block_size. Marek Vavruša #3396 +* Функции `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64` теперь работают для произвольного количества аргументов, а также для аргументов-кортежей. #3451 #3519 +* Функция `arrayReverse` теперь работает с любыми типами массивов. 73e3a7b662161d6005e7727d8a711b930386b871 +* Добавлен опциональный параметр - размер слота для функции `timeSlots`. Kirill Shvakov #3724 +* Для `FULL` и `RIGHT JOIN` учитывается настройка `max_block_size` для потока неприсоединённых данных из правой таблицы. Amos Bird #3699 +* В `clickhouse-benchmark` и `clickhouse-performance-test` добавлен параметр командной строки `--secure` для включения TLS. #3688 #3690 +* Преобразование типов в случае, если структура таблицы типа `Buffer` не соответствует структуре таблицы назначения. Vitaly Baranov #3603 +* Добавлена настройка `tcp_keep_alive_timeout` для включения keep-alive пакетов после неактивности в течение указанного интервала времени. #3441 +* Убрано излишнее квотирование значений ключа партиции в таблице `system.parts`, если он состоит из одного столбца. #3652 +* Функция деления с остатком работает для типов данных `Date` и `DateTime`. #3385 +* Добавлены синонимы функций `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, `MID`. #3774 #3763 Некоторые имена функций сделаны регистронезависимыми для совместимости со стандартом SQL. Добавлен синтаксический сахар `SUBSTRING(expr FROM start FOR length)` для совместимости с SQL. #3804 +* Добавлена возможность фиксации (`mlock`) страниц памяти, соответствующих исполняемому коду `clickhouse-server` для предотвращения вытеснения их из памяти. Возможность выключена по-умолчанию. #3553 +* Увеличена производительность чтения с `O_DIRECT` (с включенной опцией `min_bytes_to_use_direct_io`). #3405 +* Улучшена производительность работы функции `dictGet...OrDefault` в случае константного аргумента-ключа и неконстантного аргумента-default. Amos Bird #3563 +* В функции `firstSignificantSubdomain` добавлена обработка доменов `gov`, `mil`, `edu`. #3601 Увеличена производительность работы. #3628 * Возможность указания произвольных переменных окружения для запуска `clickhouse-server` посредством `SYS-V init.d`-скрипта с помощью указания `CLICKHOUSE_PROGRAM_ENV` в `/etc/default/clickhouse`. -* Правильный код возврата init-скрипта clickhouse-server. -* В таблицу `system.metrics` добавлена метрика `VersionInteger`, а в `system.build_options` добавлена строчка `VERSION_INTEGER`, содержащая версию ClickHouse в числовом представлении, вида `18016000`. -* Удалена возможность сравнения типа `Date` с числом, чтобы избежать потенциальных ошибок вида `date = 2018-12-17`, где ошибочно не указаны кавычки вокруг даты. -* Исправлено поведение функций с состоянием типа `rowNumberInAllBlocks` - раньше они выдавали число на единицу больше вследствие их запуска во время анализа запроса. -* При невозможности удалить файл `force_restore_data`, выводится сообщение об ошибке. +Pavlo Bashynskyi #3612 +* Правильный код возврата init-скрипта clickhouse-server. #3516 +* В таблицу `system.metrics` добавлена метрика `VersionInteger`, а в `system.build_options` добавлена строчка `VERSION_INTEGER`, содержащая версию ClickHouse в числовом представлении, вида `18016000`. #3644 +* Удалена возможность сравнения типа `Date` с числом, чтобы избежать потенциальных ошибок вида `date = 2018-12-17`, где ошибочно не указаны кавычки вокруг даты. #3687 +* Исправлено поведение функций с состоянием типа `rowNumberInAllBlocks` - раньше они выдавали число на единицу больше вследствие их запуска во время анализа запроса. Amos Bird #3729 +* При невозможности удалить файл `force_restore_data`, выводится сообщение об ошибке. Amos Bird #3794 ### Улучшение сборки: -* Обновлена библиотека `jemalloc`, что исправляет потенциальную утечку памяти. -* Для debug сборок включено по-умолчанию профилирование `jemalloc`. -* Добавлена возможность запуска интеграционных тестов, при наличии установленным в системе лишь `Docker`. -* Добавлен fuzz тест выражений в SELECT запросах. -* Добавлен покоммитный стресс-тест, выполняющий функциональные тесты параллельно и в произвольном порядке, позволяющий обнаружить больше race conditions. -* Улучшение способа запуска clickhouse-server в Docker образе. -* Для Docker образа добавлена поддержка инициализации базы данных с помощью файлов в директории `/docker-entrypoint-initdb.d`. -* Исправления для сборки под ARM. +* Обновлена библиотека `jemalloc`, что исправляет потенциальную утечку памяти. Amos Bird #3557 +* Для debug сборок включено по-умолчанию профилирование `jemalloc`. 2cc82f5cbe266421cd4c1165286c2c47e5ffcb15 +* Добавлена возможность запуска интеграционных тестов, при наличии установленным в системе лишь `Docker`. #3650 +* Добавлен fuzz тест выражений в SELECT запросах. #3442 +* Добавлен покоммитный стресс-тест, выполняющий функциональные тесты параллельно и в произвольном порядке, позволяющий обнаружить больше race conditions. #3438 +* Улучшение способа запуска clickhouse-server в Docker образе. Elghazal Ahmed #3663 +* Для Docker образа добавлена поддержка инициализации базы данных с помощью файлов в директории `/docker-entrypoint-initdb.d`. Konstantin Lebedev #3695 +* Исправления для сборки под ARM. #3709 ### Обратно несовместимые изменения: -* Удалена возможность сравнения типа `Date` с числом, необходимо вместо `toDate('2018-12-18') = 17883`, использовать явное приведение типов `= toDate(17883)` +* Удалена возможность сравнения типа `Date` с числом, необходимо вместо `toDate('2018-12-18') = 17883`, использовать явное приведение типов `= toDate(17883)` #3687 ## ClickHouse release 18.14.18, 2018-12-04 @@ -98,7 +99,7 @@ ### Исправления ошибок: * Исправлена ошибка в функции `dictGet...` для словарей типа `range`, если один из аргументов константный, а другой - нет. [#3751](https://github.com/yandex/ClickHouse/pull/3751) * Исправлена ошибка, приводящая к выводу сообщений `netlink: '...': attribute type 1 has an invalid length` в логе ядра Linux, проявляющаяся на достаточно новых ядрах Linux. [#3749](https://github.com/yandex/ClickHouse/pull/3749) -* Исправлен segfault при выполнении функции `empty` от аргумента типа `FixedString`. [#3703](https://github.com/yandex/ClickHouse/pull/3703) +* Исправлен segfault при выполнении функции `empty` от аргумента типа `FixedString`. [Daniel, Dao Quang Minh](https://github.com/yandex/ClickHouse/pull/3703) * Исправлена избыточная аллокация памяти при большом значении настройки `max_query_size` (кусок памяти размера `max_query_size` выделялся сразу). [#3720](https://github.com/yandex/ClickHouse/pull/3720) ### Улучшения процесса сборки ClickHouse: From c47dfb064e0122cb353b9c9df438f0c6f7deb26d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 19 Dec 2018 03:19:31 +0300 Subject: [PATCH 061/181] Update ExecuteScalarSubqueriesVisitor.cpp --- dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 7a43d7362e6..59a8c084b9e 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -102,7 +102,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr { if (typeid_cast(block.safeGetByPosition(0).type.get())) { - throw Exception("Scalar subquery can't contain an aggregate function", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + throw Exception("Scalar subquery can't return an aggregate function state", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); } auto lit = std::make_unique((*block.safeGetByPosition(0).column)[0]); @@ -125,7 +125,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr { if (typeid_cast(block.safeGetByPosition(i).type.get())) { - throw Exception("Scalar subquery can't contain an aggregate function", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + throw Exception("Scalar subquery can't return an aggregate function state", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); } exp_list->children[i] = addTypeConversion( From b9a68fbe938c3ec2a83fc56adb05d2bb111e12a0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 04:29:40 +0300 Subject: [PATCH 062/181] Merging #3721; Added a test --- dbms/src/Functions/FunctionsRound.h | 65 ++++++------------- .../00161_rounding_functions.reference | 11 ---- .../0_stateless/00161_rounding_functions.sql | 3 - 3 files changed, 21 insertions(+), 58 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index bb29b92471a..ee9eb209adc 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } @@ -562,6 +563,9 @@ public: }; +/** Rounds down to a number within explicitly specified array. + * If the value is less than the minimal bound - returns the minimal bound. + */ class FunctionRoundDown : public IFunction { public: @@ -574,6 +578,7 @@ public: bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override @@ -601,7 +606,7 @@ public: return getLeastSupertype({type_x, type_arr_nested}); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override { const ColumnConst * array = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); if (!array) @@ -612,12 +617,6 @@ public: auto in_column = block.getByPosition(arguments[0]).column; const auto & in_type = block.getByPosition(arguments[0]).type; - if (in_column->isColumnConst()) - { - executeConst(block, arguments, result, input_rows_count); - return; - } - const auto & return_type = block.getByPosition(result).type; auto column_result = return_type->createColumn(); auto out = column_result.get(); @@ -629,10 +628,12 @@ public: const auto in = in_column.get(); auto boundaries = array->getValue(); - for (size_t i = 0; i < boundaries.size(); ++i) - { + size_t num_boundaries = boundaries.size(); + if (!num_boundaries) + throw Exception("Empty array is illegal for boundaries in " + getName() + " function", ErrorCodes::BAD_ARGUMENTS); + + for (size_t i = 0; i < num_boundaries; ++i) boundaries[i] = convertFieldToType(boundaries[i], *return_type); - } if (!executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) @@ -652,31 +653,6 @@ public: } private: - void executeConst(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) - { - /// Materialize the input column and compute the function as usual. - - Block tmp_block; - ColumnNumbers tmp_arguments; - - tmp_block.insert(block.getByPosition(arguments[0])); - tmp_block.getByPosition(0).column = tmp_block.getByPosition(0).column->cloneResized(input_rows_count)->convertToFullColumnIfConst(); - tmp_arguments.push_back(0); - - for (size_t i = 1; i < arguments.size(); ++i) - { - tmp_block.insert(block.getByPosition(arguments[i])); - tmp_arguments.push_back(i); - } - - tmp_block.insert(block.getByPosition(result)); - size_t tmp_result = arguments.size(); - - execute(tmp_block, tmp_arguments, tmp_result, input_rows_count); - - block.getByPosition(result).column = tmp_block.getByPosition(tmp_result).column; - } - template bool executeNum(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) { @@ -693,24 +669,25 @@ private: template void executeImplNumToNum(const PaddedPODArray & src, PaddedPODArray & dst, const Array & boundaries) { - PaddedPODArray bvalues(boundaries.size()); + std::vector boundary_values(boundaries.size()); for (size_t i = 0; i < boundaries.size(); ++i) - { - bvalues[i] = boundaries[i].get(); - } + boundary_values[i] = boundaries[i].get(); + + std::sort(boundary_values.begin(), boundary_values.end()); + boundary_values.erase(std::unique(boundary_values.begin(), boundary_values.end()), boundary_values.end()); size_t size = src.size(); dst.resize(size); for (size_t i = 0; i < size; ++i) { - auto it = std::upper_bound(bvalues.begin(), bvalues.end(), src[i]); - if (it == bvalues.end()) + auto it = std::upper_bound(boundary_values.begin(), boundary_values.end(), src[i]); + if (it == boundary_values.end()) { - dst[i] = bvalues.back(); + dst[i] = boundary_values.back(); } - else if (it == bvalues.begin()) + else if (it == boundary_values.begin()) { - dst[i] = bvalues.front(); + dst[i] = boundary_values.front(); } else { diff --git a/dbms/tests/queries/0_stateless/00161_rounding_functions.reference b/dbms/tests/queries/0_stateless/00161_rounding_functions.reference index fbfcaa2b754..9fa1965657e 100644 --- a/dbms/tests/queries/0_stateless/00161_rounding_functions.reference +++ b/dbms/tests/queries/0_stateless/00161_rounding_functions.reference @@ -682,14 +682,3 @@ 12345.6789 12340 12300 12000 10000 0 12345.6 12345.67 12345.678 12345.6789 12345.6789 64 64 2 0 0 0.5 0 -0.5 -0.5 -0.125 -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 5 -7 5 -8 5 -9 5 -10 5 diff --git a/dbms/tests/queries/0_stateless/00161_rounding_functions.sql b/dbms/tests/queries/0_stateless/00161_rounding_functions.sql index 0b8c164bb4f..4930d9d0ace 100644 --- a/dbms/tests/queries/0_stateless/00161_rounding_functions.sql +++ b/dbms/tests/queries/0_stateless/00161_rounding_functions.sql @@ -43,6 +43,3 @@ SELECT 12345.6789 AS x, floor(x, -1), floor(x, -2), floor(x, -3), floor(x, -4), SELECT roundToExp2(100), roundToExp2(64), roundToExp2(3), roundToExp2(0), roundToExp2(-1); SELECT roundToExp2(0.9), roundToExp2(0), roundToExp2(-0.5), roundToExp2(-0.6), roundToExp2(-0.2); - - -SELECT toInt32(number) as x, roundDown(x, [0, 1, 2, 3, 4, 5]) FROM system.numbers LIMIT 10; From 4a6c25e65d9cec64428db4a067be9171fdef2d3e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 04:30:32 +0300 Subject: [PATCH 063/181] Merging #3721; Added a test --- .../0_stateless/00805_round_down.reference | 53 +++++++++++++++++++ .../queries/0_stateless/00805_round_down.sql | 14 +++++ 2 files changed, 67 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00805_round_down.reference create mode 100644 dbms/tests/queries/0_stateless/00805_round_down.sql diff --git a/dbms/tests/queries/0_stateless/00805_round_down.reference b/dbms/tests/queries/0_stateless/00805_round_down.reference new file mode 100644 index 00000000000..7cd6dc5b00c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00805_round_down.reference @@ -0,0 +1,53 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 5 +7 5 +8 5 +9 5 +0 -1.5 +1 -1.5 +2 -1.5 +3 2.718281828459045 +4 3.141592653589793 +5 3.141592653589793 +6 5.5 +7 5.5 +8 5.5 +9 5.5 +0 2.718281828459045 +1 2.718281828459045 +2 2.718281828459045 +3 2.718281828459045 +4 3.141592653589793 +5 3.141592653589793 +6 3.141592653589793 +7 3.141592653589793 +8 3.141592653589793 +9 3.141592653589793 +0 4 +1 4 +2 4 +3 4 +4 4 +5 5 +6 6 +7 6 +8 6 +9 6 +1 4 +1 1 +1 1.5 +0 0 +1 0 +2 1.25 +3 2.5 +4 3.75 +5 5 +6 5 +7 6.25 +8 7.5 +9 8.75 diff --git a/dbms/tests/queries/0_stateless/00805_round_down.sql b/dbms/tests/queries/0_stateless/00805_round_down.sql new file mode 100644 index 00000000000..bfd0843a7cb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00805_round_down.sql @@ -0,0 +1,14 @@ +SELECT number as x, roundDown(x, [0, 1, 2, 3, 4, 5]) FROM system.numbers LIMIT 10; +SELECT toUInt8(number) as x, roundDown(x, [-1.5, e(), pi(), 5.5]) FROM system.numbers LIMIT 10; +SELECT toInt32(number) as x, roundDown(x, [e(), pi(), pi(), e()]) FROM system.numbers LIMIT 10; +SELECT number as x, roundDown(x, [6, 5, 4]) FROM system.numbers LIMIT 10; +SELECT 1 as x, roundDown(x, [6, 5, 4]); + +SET send_logs_level = 'none'; +SELECT 1 as x, roundDown(x, []); -- { serverError 43 } +SELECT 1 as x, roundDown(x, emptyArrayUInt8()); -- { serverError 44 } + +SELECT 1 as x, roundDown(x, [1]); +SELECT 1 as x, roundDown(x, [1.5]); + +SELECT number % 10 as x, roundDown(x, (SELECT groupArray(number * 1.25) FROM numbers(100000))) FROM system.numbers LIMIT 10; From 9c23556cb095eea105ceef461838824c279f7925 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 05:11:31 +0300 Subject: [PATCH 064/181] Basic implementation for Decimal #3721 --- dbms/src/Functions/FunctionsRound.h | 31 ++++++++++++++----- .../0_stateless/00805_round_down.reference | 10 ++++++ .../queries/0_stateless/00805_round_down.sql | 3 ++ 3 files changed, 36 insertions(+), 8 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index ee9eb209adc..92d70633c12 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -585,7 +585,7 @@ public: { const DataTypePtr & type_x = arguments[0]; - if (!isNumber(type_x)) + if (!(isNumber(type_x) || isDecimal(type_x))) throw Exception{"Unsupported type " + type_x->getName() + " of first argument of function " + getName() + ", must be numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -598,7 +598,7 @@ public: const auto type_arr_nested = type_arr->getNestedType(); - if (!isNumber(type_arr_nested)) + if (!(isNumber(type_arr_nested) || isDecimal(type_arr_nested))) { throw Exception{"Elements of array of second argument of function " + getName() + " must be numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; @@ -644,7 +644,10 @@ public: && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) - && !executeNum(in, out, boundaries)) + && !executeNum(in, out, boundaries) + && !executeDecimal(in, out, boundaries) + && !executeDecimal(in, out, boundaries) + && !executeDecimal(in, out, boundaries)) { throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } @@ -659,19 +662,31 @@ private: const auto in = checkAndGetColumn>(in_untyped); auto out = typeid_cast *>(out_untyped); if (!in || !out) - { return false; - } + executeImplNumToNum(in->getData(), out->getData(), boundaries); return true; } template - void executeImplNumToNum(const PaddedPODArray & src, PaddedPODArray & dst, const Array & boundaries) + bool executeDecimal(const IColumn * in_untyped, IColumn * out_untyped, const Array & boundaries) { - std::vector boundary_values(boundaries.size()); + const auto in = checkAndGetColumn>(in_untyped); + auto out = typeid_cast *>(out_untyped); + if (!in || !out) + return false; + + executeImplNumToNum(in->getData(), out->getData(), boundaries); + return true; + } + + template + void executeImplNumToNum(const Container & src, Container & dst, const Array & boundaries) + { + using ValueType = typename Container::value_type; + std::vector boundary_values(boundaries.size()); for (size_t i = 0; i < boundaries.size(); ++i) - boundary_values[i] = boundaries[i].get(); + boundary_values[i] = boundaries[i].get(); std::sort(boundary_values.begin(), boundary_values.end()); boundary_values.erase(std::unique(boundary_values.begin(), boundary_values.end()), boundary_values.end()); diff --git a/dbms/tests/queries/0_stateless/00805_round_down.reference b/dbms/tests/queries/0_stateless/00805_round_down.reference index 7cd6dc5b00c..b37e8c8f1a0 100644 --- a/dbms/tests/queries/0_stateless/00805_round_down.reference +++ b/dbms/tests/queries/0_stateless/00805_round_down.reference @@ -51,3 +51,13 @@ 7 6.25 8 7.5 9 8.75 +0.00000 4.00000 +0.01000 4.00000 +0.02000 4.00000 +0.03000 4.00000 +0.04000 4.00000 +0.05000 4.00000 +0.06000 4.00000 +0.07000 4.00000 +0.08000 4.00000 +0.09000 4.00000 diff --git a/dbms/tests/queries/0_stateless/00805_round_down.sql b/dbms/tests/queries/0_stateless/00805_round_down.sql index bfd0843a7cb..0ba4804031c 100644 --- a/dbms/tests/queries/0_stateless/00805_round_down.sql +++ b/dbms/tests/queries/0_stateless/00805_round_down.sql @@ -12,3 +12,6 @@ SELECT 1 as x, roundDown(x, [1]); SELECT 1 as x, roundDown(x, [1.5]); SELECT number % 10 as x, roundDown(x, (SELECT groupArray(number * 1.25) FROM numbers(100000))) FROM system.numbers LIMIT 10; + +SELECT toDecimal64(number, 5) / 100 as x, roundDown(x, [4, 5, 6]) FROM system.numbers LIMIT 10; + From 689de0289f29d82a8f617abe0bb0f02af2f66f57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 05:37:20 +0300 Subject: [PATCH 065/181] Added performance test #3721 --- .../tests/performance/roundDown/roundDown.xml | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 dbms/tests/performance/roundDown/roundDown.xml diff --git a/dbms/tests/performance/roundDown/roundDown.xml b/dbms/tests/performance/roundDown/roundDown.xml new file mode 100644 index 00000000000..18af6916d40 --- /dev/null +++ b/dbms/tests/performance/roundDown/roundDown.xml @@ -0,0 +1,24 @@ + + roundDown + once + + + + 10000 + + + 5000 + 20000 + + + + + + + + SELECT count() FROM system.numbers WHERE NOT ignore(roundDuration(rand() % 65536)) + SELECT count() FROM system.numbers WHERE NOT ignore(roundDown(rand() % 65536, [0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000])) + SELECT count() FROM system.numbers WHERE NOT ignore(roundAge(rand() % 100)) + SELECT count() FROM system.numbers WHERE NOT ignore(roundDown(rand() % 100, [0, 1, 18, 25, 35, 45, 55])) + SELECT count() FROM system.numbers WHERE NOT ignore(roundDown(rand() % 65536, (SELECT groupArray(number) FROM numbers(65536)))) + From 36836faf626db96f911547fe74676ba1248f6977 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 05:43:09 +0300 Subject: [PATCH 066/181] Full support for Decimal #3721 --- dbms/src/DataTypes/FieldToDataType.cpp | 1 - dbms/src/Functions/FunctionsRound.h | 20 +++++++------------ .../0_stateless/00805_round_down.reference | 20 +++++++++++++++++++ .../queries/0_stateless/00805_round_down.sql | 4 +++- 4 files changed, 30 insertions(+), 15 deletions(-) diff --git a/dbms/src/DataTypes/FieldToDataType.cpp b/dbms/src/DataTypes/FieldToDataType.cpp index 3086c9167c2..18fa0e97b88 100644 --- a/dbms/src/DataTypes/FieldToDataType.cpp +++ b/dbms/src/DataTypes/FieldToDataType.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 92d70633c12..3c9be5f15b9 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -608,33 +607,28 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override { - const ColumnConst * array = checkAndGetColumnConst(block.getByPosition(arguments[1]).column.get()); - if (!array) - { - throw Exception{"Second argument of function " + getName() + " must be constant array.", ErrorCodes::ILLEGAL_COLUMN}; - } - auto in_column = block.getByPosition(arguments[0]).column; const auto & in_type = block.getByPosition(arguments[0]).type; + auto array_column = block.getByPosition(arguments[1]).column; + const auto & array_type = block.getByPosition(arguments[1]).type; + const auto & return_type = block.getByPosition(result).type; auto column_result = return_type->createColumn(); auto out = column_result.get(); if (!in_type->equals(*return_type)) - { in_column = castColumn(block.getByPosition(arguments[0]), return_type, context); - } + + if (!array_type->equals(*return_type)) + array_column = castColumn(block.getByPosition(arguments[1]), std::make_shared(return_type), context); const auto in = in_column.get(); - auto boundaries = array->getValue(); + auto boundaries = typeid_cast(*array_column).getValue(); size_t num_boundaries = boundaries.size(); if (!num_boundaries) throw Exception("Empty array is illegal for boundaries in " + getName() + " function", ErrorCodes::BAD_ARGUMENTS); - for (size_t i = 0; i < num_boundaries; ++i) - boundaries[i] = convertFieldToType(boundaries[i], *return_type); - if (!executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) && !executeNum(in, out, boundaries) diff --git a/dbms/tests/queries/0_stateless/00805_round_down.reference b/dbms/tests/queries/0_stateless/00805_round_down.reference index b37e8c8f1a0..0ed04c7757f 100644 --- a/dbms/tests/queries/0_stateless/00805_round_down.reference +++ b/dbms/tests/queries/0_stateless/00805_round_down.reference @@ -61,3 +61,23 @@ 0.07000 4.00000 0.08000 4.00000 0.09000 4.00000 +0.00000 0.04000 +0.01000 0.04000 +0.02000 0.04000 +0.03000 0.04000 +0.04000 0.04000 +0.05000 0.05000 +0.06000 0.06000 +0.07000 0.06000 +0.08000 0.06000 +0.09000 0.06000 +0.00000 0.04000 +0.01000 0.04000 +0.02000 0.04000 +0.03000 0.04000 +0.04000 0.04000 +0.05000 0.05000 +0.06000 0.06000 +0.07000 0.06000 +0.08000 0.06000 +0.09000 0.06000 diff --git a/dbms/tests/queries/0_stateless/00805_round_down.sql b/dbms/tests/queries/0_stateless/00805_round_down.sql index 0ba4804031c..bf25f7be952 100644 --- a/dbms/tests/queries/0_stateless/00805_round_down.sql +++ b/dbms/tests/queries/0_stateless/00805_round_down.sql @@ -7,6 +7,7 @@ SELECT 1 as x, roundDown(x, [6, 5, 4]); SET send_logs_level = 'none'; SELECT 1 as x, roundDown(x, []); -- { serverError 43 } SELECT 1 as x, roundDown(x, emptyArrayUInt8()); -- { serverError 44 } +SELECT roundDown(number, [number]) FROM system.numbers LIMIT 10; -- { serverError 44 } SELECT 1 as x, roundDown(x, [1]); SELECT 1 as x, roundDown(x, [1.5]); @@ -14,4 +15,5 @@ SELECT 1 as x, roundDown(x, [1.5]); SELECT number % 10 as x, roundDown(x, (SELECT groupArray(number * 1.25) FROM numbers(100000))) FROM system.numbers LIMIT 10; SELECT toDecimal64(number, 5) / 100 as x, roundDown(x, [4, 5, 6]) FROM system.numbers LIMIT 10; - +SELECT toDecimal64(number, 5) / 100 as x, roundDown(x, [toDecimal64(0.04, 5), toDecimal64(0.05, 5), toDecimal64(0.06, 5)]) FROM system.numbers LIMIT 10; +SELECT toDecimal64(number, 5) / 100 as x, roundDown(x, [toDecimal32(0.04, 2), toDecimal32(0.05, 2), toDecimal32(0.06, 2)]) FROM system.numbers LIMIT 10; From 159e0a23b657186e820fce0ec6e8fe371bf77cdb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 06:34:42 +0300 Subject: [PATCH 067/181] Added links [#CLICKHOUSE-2] --- CHANGELOG_RU.md | 152 ++++++++++++++++++++++++------------------------ 1 file changed, 76 insertions(+), 76 deletions(-) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index ff36f3e7d64..86ea3544a19 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -2,96 +2,96 @@ ### Новые возможности: -* Вычисление `DEFAULT` выражений для отсутствующих полей при загрузке данных в полуструктурированных форматах (`JSONEachRow`, `TSKV`). #3555 -* Для запроса `ALTER TABLE` добавлено действие `MODIFY ORDER BY` для изменения ключа сортировки при одновременном добавлении или удалении столбца таблицы. Это полезно для таблиц семейства `MergeTree`, выполняющих дополнительную работу при слияниях, согласно этому ключу сортировки, как например, `SummingMergeTree`, `AggregatingMergeTree` и т. п. #3581 #3755 -* Для таблиц семейства `MergeTree` появилась возможность указать различный ключ сортировки (`ORDER BY`) и индекс (`PRIMARY KEY`). Ключ сортировки может быть длиннее, чем индекс. #3581 -* Добавлена табличная функция `hdfs` и движок таблиц `HDFS` для импорта и экспорта данных в HDFS. chenxing-xc #3617 -* Добавлены функции для работы с base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. Alexander Krasheninnikov #3350 -* Для агрегатной функции `uniqCombined` появилась возможность настраивать точность работы с помощью параметра (выбирать количество ячеек HyperLogLog). #3406 -* Добавлена таблица `system.contributors`, содержащая имена всех, кто делал коммиты в ClickHouse. #3452 -* Добавлена возможность не указывать партицию для запроса `ALTER TABLE ... FREEZE` для бэкапа сразу всех партиций. #3514 -* Добавлены функции `dictGet`, `dictGetOrDefault` без указания типа возвращаемого значения. Тип определяется автоматически из описания словаря. Amos Bird #3564 -* Возможность указания комментария для столбца в описании таблицы и изменения его с помощью `ALTER`. #3377 -* Возможность чтения из таблицы типа `Join` в случае простых ключей. Amos Bird #3728 -* Возможность указания настроек `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, `join_overflow_mode` при создании таблицы типа `Join`. Amos Bird #3728 -* Добавлена функция `joinGet`, позволяющая использовать таблицы типа `Join` как словарь. Amos Bird #3728 -* Добавлены столбцы `partition_key`, `sorting_key`, `primary_key`, `sampling_key` в таблицу `system.tables`, позволяющие получить информацию о ключах таблицы. #3609 -* Добавлены столбцы `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, `is_in_sampling_key` в таблицу `system.columns`. #3609 -* Добавлены столбцы `min_time`, `max_time` в таблицу `system.parts`. Эти столбцы заполняются, если ключ партиционирования является выражением от столбцов типа `DateTime`. Emmanuel Donin de Rosière #3800 +* Вычисление `DEFAULT` выражений для отсутствующих полей при загрузке данных в полуструктурированных форматах (`JSONEachRow`, `TSKV`). [#3555](https://github.com/yandex/ClickHouse/pull/3555) +* Для запроса `ALTER TABLE` добавлено действие `MODIFY ORDER BY` для изменения ключа сортировки при одновременном добавлении или удалении столбца таблицы. Это полезно для таблиц семейства `MergeTree`, выполняющих дополнительную работу при слияниях, согласно этому ключу сортировки, как например, `SummingMergeTree`, `AggregatingMergeTree` и т. п. [#3581](https://github.com/yandex/ClickHouse/pull/3581) [#3755](https://github.com/yandex/ClickHouse/pull/3755) +* Для таблиц семейства `MergeTree` появилась возможность указать различный ключ сортировки (`ORDER BY`) и индекс (`PRIMARY KEY`). Ключ сортировки может быть длиннее, чем индекс. [#3581](https://github.com/yandex/ClickHouse/pull/3581) +* Добавлена табличная функция `hdfs` и движок таблиц `HDFS` для импорта и экспорта данных в HDFS. chenxing-xc [#3617](https://github.com/yandex/ClickHouse/pull/3617) +* Добавлены функции для работы с base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. [Alexander Krasheninnikov](https://github.com/yandex/ClickHouse/pull/3350) +* Для агрегатной функции `uniqCombined` появилась возможность настраивать точность работы с помощью параметра (выбирать количество ячеек HyperLogLog). [#3406](https://github.com/yandex/ClickHouse/pull/3406) +* Добавлена таблица `system.contributors`, содержащая имена всех, кто делал коммиты в ClickHouse. [#3452](https://github.com/yandex/ClickHouse/pull/3452) +* Добавлена возможность не указывать партицию для запроса `ALTER TABLE ... FREEZE` для бэкапа сразу всех партиций. [#3514](https://github.com/yandex/ClickHouse/pull/3514) +* Добавлены функции `dictGet`, `dictGetOrDefault` без указания типа возвращаемого значения. Тип определяется автоматически из описания словаря. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3564) +* Возможность указания комментария для столбца в описании таблицы и изменения его с помощью `ALTER`. [#3377](https://github.com/yandex/ClickHouse/pull/3377) +* Возможность чтения из таблицы типа `Join` в случае простых ключей. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3728) +* Возможность указания настроек `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, `join_overflow_mode` при создании таблицы типа `Join`. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3728) +* Добавлена функция `joinGet`, позволяющая использовать таблицы типа `Join` как словарь. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3728) +* Добавлены столбцы `partition_key`, `sorting_key`, `primary_key`, `sampling_key` в таблицу `system.tables`, позволяющие получить информацию о ключах таблицы. [#3609](https://github.com/yandex/ClickHouse/pull/3609) +* Добавлены столбцы `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, `is_in_sampling_key` в таблицу `system.columns`. [#3609](https://github.com/yandex/ClickHouse/pull/3609) +* Добавлены столбцы `min_time`, `max_time` в таблицу `system.parts`. Эти столбцы заполняются, если ключ партиционирования является выражением от столбцов типа `DateTime`. [Emmanuel Donin de Rosière](https://github.com/yandex/ClickHouse/pull/3800) ### Исправления ошибок: -* Исправления и улучшения производительности для типа данных `LowCardinality`. `GROUP BY` по `LowCardinality(Nullable(...))`. Получение `extremes` значений. Выполнение функций высшего порядка. `LEFT ARRAY JOIN`. Распределённый `GROUP BY`. Функции, возвращающие `Array`. Выполнение `ORDER BY`. Запись в `Distributed` таблицы (nicelulu). Обратная совместимость для запросов `INSERT` от старых клиентов, реализующих `Native` протокол. Поддержка `LowCardinality` для `JOIN`. Производительность при работе в один поток. #3823 #3803 #3799 #3769 #3744 #3681 #3651 #3649 #3641 #3632 #3568 #3523 #3518 -* Исправлена работа настройки `select_sequential_consistency`. Ранее, при включенной настройке, после начала записи в новую партицию, мог возвращаться неполный результат. #2863 -* Корректное указание базы данных при выполнении DDL запросов `ON CLUSTER`, а также при выполнении `ALTER UPDATE/DELETE`. #3772 #3460 -* Корректное указание базы данных для подзапросов внутри VIEW. #3521 -* Исправлена работа `PREWHERE` с `FINAL` для `VersionedCollapsingMergeTree`. 7167bfd7b365538f7a91c4307ad77e552ab4e8c1 -* Возможность с помощью запроса `KILL QUERY` отмены запросов, которые ещё не начали выполняться из-за ожидания блокировки таблицы. #3517 -* Исправлены расчёты с датой и временем в случае, если стрелки часов были переведены назад в полночь (это происходит в Иране, а также было Москве с 1981 по 1983 год). Ранее это приводило к тому, что стрелки часов переводились на сутки раньше, чем нужно, а также приводило к некорректному форматированию даты-с-временем в текстовом виде. #3819 -* Исправлена работа некоторых случаев `VIEW` и подзапросов без указания базы данных. Winter Zhang #3521 -* Исправлен race condition при одновременном чтении из `MATERIALIZED VIEW` и удалением `MATERIALIZED VIEW` из-за отсутствия блокировки внутренней таблицы `MATERIALIZED VIEW`. #3404 #3694 -* Исправлена ошибка `Lock handler cannot be nullptr.` #3689 -* Исправления выполнения запросов при включенной настройке `compile_expressions` (выключена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. #3457 +* Исправления и улучшения производительности для типа данных `LowCardinality`. `GROUP BY` по `LowCardinality(Nullable(...))`. Получение `extremes` значений. Выполнение функций высшего порядка. `LEFT ARRAY JOIN`. Распределённый `GROUP BY`. Функции, возвращающие `Array`. Выполнение `ORDER BY`. Запись в `Distributed` таблицы (nicelulu). Обратная совместимость для запросов `INSERT` от старых клиентов, реализующих `Native` протокол. Поддержка `LowCardinality` для `JOIN`. Производительность при работе в один поток. [#3823](https://github.com/yandex/ClickHouse/pull/3823) [#3803](https://github.com/yandex/ClickHouse/pull/3803) [#3799](https://github.com/yandex/ClickHouse/pull/3799) [#3769](https://github.com/yandex/ClickHouse/pull/3769) [#3744](https://github.com/yandex/ClickHouse/pull/3744) [#3681](https://github.com/yandex/ClickHouse/pull/3681) [#3651](https://github.com/yandex/ClickHouse/pull/3651) [#3649](https://github.com/yandex/ClickHouse/pull/3649) [#3641](https://github.com/yandex/ClickHouse/pull/3641) [#3632](https://github.com/yandex/ClickHouse/pull/3632) [#3568](https://github.com/yandex/ClickHouse/pull/3568) [#3523](https://github.com/yandex/ClickHouse/pull/3523) [#3518](https://github.com/yandex/ClickHouse/pull/3518) +* Исправлена работа настройки `select_sequential_consistency`. Ранее, при включенной настройке, после начала записи в новую партицию, мог возвращаться неполный результат. [#2863](https://github.com/yandex/ClickHouse/pull/2863) +* Корректное указание базы данных при выполнении DDL запросов `ON CLUSTER`, а также при выполнении `ALTER UPDATE/DELETE`. [#3772](https://github.com/yandex/ClickHouse/pull/3772) [#3460](https://github.com/yandex/ClickHouse/pull/3460) +* Корректное указание базы данных для подзапросов внутри VIEW. [#3521](https://github.com/yandex/ClickHouse/pull/3521) +* Исправлена работа `PREWHERE` с `FINAL` для `VersionedCollapsingMergeTree`. [7167bfd7](https://github.com/yandex/ClickHouse/commit/7167bfd7b365538f7a91c4307ad77e552ab4e8c1) +* Возможность с помощью запроса `KILL QUERY` отмены запросов, которые ещё не начали выполняться из-за ожидания блокировки таблицы. [#3517](https://github.com/yandex/ClickHouse/pull/3517) +* Исправлены расчёты с датой и временем в случае, если стрелки часов были переведены назад в полночь (это происходит в Иране, а также было Москве с 1981 по 1983 год). Ранее это приводило к тому, что стрелки часов переводились на сутки раньше, чем нужно, а также приводило к некорректному форматированию даты-с-временем в текстовом виде. [#3819](https://github.com/yandex/ClickHouse/pull/3819) +* Исправлена работа некоторых случаев `VIEW` и подзапросов без указания базы данных. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/3521) +* Исправлен race condition при одновременном чтении из `MATERIALIZED VIEW` и удалением `MATERIALIZED VIEW` из-за отсутствия блокировки внутренней таблицы `MATERIALIZED VIEW`. [#3404](https://github.com/yandex/ClickHouse/pull/3404) [#3694](https://github.com/yandex/ClickHouse/pull/3694) +* Исправлена ошибка `Lock handler cannot be nullptr.` [#3689](https://github.com/yandex/ClickHouse/pull/3689) +* Исправления выполнения запросов при включенной настройке `compile_expressions` (выключена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. [#3457](https://github.com/yandex/ClickHouse/pull/3457) * Исправлено падение при указании неконстантного аргумента scale в функциях `toDecimal32/64/128`. -* Исправлена ошибка при попытке вставки в формате `Values` массива с `NULL` элементами в столбец типа `Array` без `Nullable` (в случае `input_format_values_interpret_expressions` = 1). #3487 #3503 -* Исправлено непрерывное логгирование ошибок в `DDLWorker`, если ZooKeeper недоступен. 8f50c620334988b28018213ec0092fe6423847e2 -* Исправлен тип возвращаемого значения для функций `quantile*` от аргументов типа `Date` и `DateTime`. #3580 -* Исправлена работа секции `WITH`, если она задаёт простой алиас без выражений. #3570 -* Исправлена обработка запросов с именованными подзапросами и квалифицированными именами столбцов при включенной настройке `enable_optimize_predicate_expression`. Winter Zhang #3588 -* Исправлена ошибка `Attempt to attach to nullptr thread group` при работе материализованных представлений. Marek Vavruša #3623 -* Исправлено падение при передаче некоторых некорректных аргументов в функцию `arrayReverse`. 73e3a7b662161d6005e7727d8a711b930386b871 -* Исправлен buffer overflow в функции `extractURLParameter`. Увеличена производительность. Добавлена корректная обработка строк, содержащих нулевые байты. 141e9799e49201d84ea8e951d1bed4fb6d3dacb5 -* Исправлен buffer overflow в функциях `lowerUTF8`, `upperUTF8`. Удалена возможность выполнения этих функций над аргументами типа `FixedString`. #3662 -* Исправлен редкий race condition при удалении таблиц типа `MergeTree`. #3680 -* Исправлен race condition при чтении из таблиц типа `Buffer` и одновременном `ALTER` либо `DROP` таблиц назначения. #3719 -* Исправлен segfault в случае превышения ограничения `max_temporary_non_const_columns`. #3788 +* Исправлена ошибка при попытке вставки в формате `Values` массива с `NULL` элементами в столбец типа `Array` без `Nullable` (в случае `input_format_values_interpret_expressions` = 1). [#3487](https://github.com/yandex/ClickHouse/pull/3487) [#3503](https://github.com/yandex/ClickHouse/pull/3503) +* Исправлено непрерывное логгирование ошибок в `DDLWorker`, если ZooKeeper недоступен. [8f50c620](https://github.com/yandex/ClickHouse/commit/8f50c620334988b28018213ec0092fe6423847e2) +* Исправлен тип возвращаемого значения для функций `quantile*` от аргументов типа `Date` и `DateTime`. [#3580](https://github.com/yandex/ClickHouse/pull/3580) +* Исправлена работа секции `WITH`, если она задаёт простой алиас без выражений. [#3570](https://github.com/yandex/ClickHouse/pull/3570) +* Исправлена обработка запросов с именованными подзапросами и квалифицированными именами столбцов при включенной настройке `enable_optimize_predicate_expression`. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/3588) +* Исправлена ошибка `Attempt to attach to nullptr thread group` при работе материализованных представлений. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3623) +* Исправлено падение при передаче некоторых некорректных аргументов в функцию `arrayReverse`. [73e3a7b6](https://github.com/yandex/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) +* Исправлен buffer overflow в функции `extractURLParameter`. Увеличена производительность. Добавлена корректная обработка строк, содержащих нулевые байты. [141e9799](https://github.com/yandex/ClickHouse/commit/141e9799e49201d84ea8e951d1bed4fb6d3dacb5) +* Исправлен buffer overflow в функциях `lowerUTF8`, `upperUTF8`. Удалена возможность выполнения этих функций над аргументами типа `FixedString`. [#3662](https://github.com/yandex/ClickHouse/pull/3662) +* Исправлен редкий race condition при удалении таблиц типа `MergeTree`. [#3680](https://github.com/yandex/ClickHouse/pull/3680) +* Исправлен race condition при чтении из таблиц типа `Buffer` и одновременном `ALTER` либо `DROP` таблиц назначения. [#3719](https://github.com/yandex/ClickHouse/pull/3719) +* Исправлен segfault в случае превышения ограничения `max_temporary_non_const_columns`. [#3788](https://github.com/yandex/ClickHouse/pull/3788) ### Улучшения: -* Обработанные конфигурационные файлы записываются сервером не в `/etc/clickhouse-server/` директорию, а в директорию `preprocessed_configs` внутри `path`. Это позволяет оставить директорию `/etc/clickhouse-server/` недоступной для записи пользователем `clickhouse`, что повышает безопасность. #2443 -* Настройка `min_merge_bytes_to_use_direct_io` выставлена по-умолчанию в 10 GiB. Слияния, образующие крупные куски таблиц семейства MergeTree, будут производиться в режиме `O_DIRECT`, что исключает вымывание кэша. #3504 -* Ускорен запуск сервера в случае наличия очень большого количества таблиц. #3398 -* Добавлен пул соединений и HTTP `Keep-Alive` для соединения между репликами. #3594 -* В случае ошибки синтаксиса запроса, в `HTTP` интерфейсе возвращается код `400 Bad Request` (ранее возвращался код 500). 31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab -* Для настройки `join_default_strictness` выбрано значение по-умолчанию `ALL` для совместимости. 120e2cbe2ff4fbad626c28042d9b28781c805afe -* Убрано логгирование в `stderr` из библиотеки `re2` в случае некорректных или сложных регулярных выражений. #3723 -* Для движка таблиц `Kafka`: проверка наличия подписок перед началом чтения из Kafka; настройка таблицы kafka_max_block_size. Marek Vavruša #3396 -* Функции `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64` теперь работают для произвольного количества аргументов, а также для аргументов-кортежей. #3451 #3519 -* Функция `arrayReverse` теперь работает с любыми типами массивов. 73e3a7b662161d6005e7727d8a711b930386b871 -* Добавлен опциональный параметр - размер слота для функции `timeSlots`. Kirill Shvakov #3724 -* Для `FULL` и `RIGHT JOIN` учитывается настройка `max_block_size` для потока неприсоединённых данных из правой таблицы. Amos Bird #3699 -* В `clickhouse-benchmark` и `clickhouse-performance-test` добавлен параметр командной строки `--secure` для включения TLS. #3688 #3690 -* Преобразование типов в случае, если структура таблицы типа `Buffer` не соответствует структуре таблицы назначения. Vitaly Baranov #3603 -* Добавлена настройка `tcp_keep_alive_timeout` для включения keep-alive пакетов после неактивности в течение указанного интервала времени. #3441 -* Убрано излишнее квотирование значений ключа партиции в таблице `system.parts`, если он состоит из одного столбца. #3652 -* Функция деления с остатком работает для типов данных `Date` и `DateTime`. #3385 -* Добавлены синонимы функций `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, `MID`. #3774 #3763 Некоторые имена функций сделаны регистронезависимыми для совместимости со стандартом SQL. Добавлен синтаксический сахар `SUBSTRING(expr FROM start FOR length)` для совместимости с SQL. #3804 -* Добавлена возможность фиксации (`mlock`) страниц памяти, соответствующих исполняемому коду `clickhouse-server` для предотвращения вытеснения их из памяти. Возможность выключена по-умолчанию. #3553 -* Увеличена производительность чтения с `O_DIRECT` (с включенной опцией `min_bytes_to_use_direct_io`). #3405 -* Улучшена производительность работы функции `dictGet...OrDefault` в случае константного аргумента-ключа и неконстантного аргумента-default. Amos Bird #3563 -* В функции `firstSignificantSubdomain` добавлена обработка доменов `gov`, `mil`, `edu`. #3601 Увеличена производительность работы. #3628 +* Обработанные конфигурационные файлы записываются сервером не в `/etc/clickhouse-server/` директорию, а в директорию `preprocessed_configs` внутри `path`. Это позволяет оставить директорию `/etc/clickhouse-server/` недоступной для записи пользователем `clickhouse`, что повышает безопасность. [#2443](https://github.com/yandex/ClickHouse/pull/2443) +* Настройка `min_merge_bytes_to_use_direct_io` выставлена по-умолчанию в 10 GiB. Слияния, образующие крупные куски таблиц семейства MergeTree, будут производиться в режиме `O_DIRECT`, что исключает вымывание кэша. [#3504](https://github.com/yandex/ClickHouse/pull/3504) +* Ускорен запуск сервера в случае наличия очень большого количества таблиц. [#3398](https://github.com/yandex/ClickHouse/pull/3398) +* Добавлен пул соединений и HTTP `Keep-Alive` для соединения между репликами. [#3594](https://github.com/yandex/ClickHouse/pull/3594) +* В случае ошибки синтаксиса запроса, в `HTTP` интерфейсе возвращается код `400 Bad Request` (ранее возвращался код 500). [31bc680a](https://github.com/yandex/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) +* Для настройки `join_default_strictness` выбрано значение по-умолчанию `ALL` для совместимости. [120e2cbe](https://github.com/yandex/ClickHouse/commit/120e2cbe2ff4fbad626c28042d9b28781c805afe) +* Убрано логгирование в `stderr` из библиотеки `re2` в случае некорректных или сложных регулярных выражений. [#3723](https://github.com/yandex/ClickHouse/pull/3723) +* Для движка таблиц `Kafka`: проверка наличия подписок перед началом чтения из Kafka; настройка таблицы kafka_max_block_size. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3396) +* Функции `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64` теперь работают для произвольного количества аргументов, а также для аргументов-кортежей. [#3451](https://github.com/yandex/ClickHouse/pull/3451) [#3519](https://github.com/yandex/ClickHouse/pull/3519) +* Функция `arrayReverse` теперь работает с любыми типами массивов. [73e3a7b6](https://github.com/yandex/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) +* Добавлен опциональный параметр - размер слота для функции `timeSlots`. [Kirill Shvakov](https://github.com/yandex/ClickHouse/pull/3724) +* Для `FULL` и `RIGHT JOIN` учитывается настройка `max_block_size` для потока неприсоединённых данных из правой таблицы. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3699) +* В `clickhouse-benchmark` и `clickhouse-performance-test` добавлен параметр командной строки `--secure` для включения TLS. [#3688](https://github.com/yandex/ClickHouse/pull/3688) [#3690](https://github.com/yandex/ClickHouse/pull/3690) +* Преобразование типов в случае, если структура таблицы типа `Buffer` не соответствует структуре таблицы назначения. [Vitaly Baranov](https://github.com/yandex/ClickHouse/pull/3603) +* Добавлена настройка `tcp_keep_alive_timeout` для включения keep-alive пакетов после неактивности в течение указанного интервала времени. [#3441](https://github.com/yandex/ClickHouse/pull/3441) +* Убрано излишнее квотирование значений ключа партиции в таблице `system.parts`, если он состоит из одного столбца. [#3652](https://github.com/yandex/ClickHouse/pull/3652) +* Функция деления с остатком работает для типов данных `Date` и `DateTime`. [#3385](https://github.com/yandex/ClickHouse/pull/3385) +* Добавлены синонимы функций `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, `MID`. [#3774](https://github.com/yandex/ClickHouse/pull/3774) [#3763](https://github.com/yandex/ClickHouse/pull/3763) Некоторые имена функций сделаны регистронезависимыми для совместимости со стандартом SQL. Добавлен синтаксический сахар `SUBSTRING(expr FROM start FOR length)` для совместимости с SQL. [#3804](https://github.com/yandex/ClickHouse/pull/3804) +* Добавлена возможность фиксации (`mlock`) страниц памяти, соответствующих исполняемому коду `clickhouse-server` для предотвращения вытеснения их из памяти. Возможность выключена по-умолчанию. [#3553](https://github.com/yandex/ClickHouse/pull/3553) +* Увеличена производительность чтения с `O_DIRECT` (с включенной опцией `min_bytes_to_use_direct_io`). [#3405](https://github.com/yandex/ClickHouse/pull/3405) +* Улучшена производительность работы функции `dictGet...OrDefault` в случае константного аргумента-ключа и неконстантного аргумента-default. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3563) +* В функции `firstSignificantSubdomain` добавлена обработка доменов `gov`, `mil`, `edu`. [Igor Hatarist](https://github.com/yandex/ClickHouse/pull/3601) Увеличена производительность работы. [#3628](https://github.com/yandex/ClickHouse/pull/3628) * Возможность указания произвольных переменных окружения для запуска `clickhouse-server` посредством `SYS-V init.d`-скрипта с помощью указания `CLICKHOUSE_PROGRAM_ENV` в `/etc/default/clickhouse`. -Pavlo Bashynskyi #3612 -* Правильный код возврата init-скрипта clickhouse-server. #3516 -* В таблицу `system.metrics` добавлена метрика `VersionInteger`, а в `system.build_options` добавлена строчка `VERSION_INTEGER`, содержащая версию ClickHouse в числовом представлении, вида `18016000`. #3644 -* Удалена возможность сравнения типа `Date` с числом, чтобы избежать потенциальных ошибок вида `date = 2018-12-17`, где ошибочно не указаны кавычки вокруг даты. #3687 -* Исправлено поведение функций с состоянием типа `rowNumberInAllBlocks` - раньше они выдавали число на единицу больше вследствие их запуска во время анализа запроса. Amos Bird #3729 -* При невозможности удалить файл `force_restore_data`, выводится сообщение об ошибке. Amos Bird #3794 +[Pavlo Bashynskyi](https://github.com/yandex/ClickHouse/pull/3612) +* Правильный код возврата init-скрипта clickhouse-server. [#3516](https://github.com/yandex/ClickHouse/pull/3516) +* В таблицу `system.metrics` добавлена метрика `VersionInteger`, а в `system.build_options` добавлена строчка `VERSION_INTEGER`, содержащая версию ClickHouse в числовом представлении, вида `18016000`. [#3644](https://github.com/yandex/ClickHouse/pull/3644) +* Удалена возможность сравнения типа `Date` с числом, чтобы избежать потенциальных ошибок вида `date = 2018-12-17`, где ошибочно не указаны кавычки вокруг даты. [#3687](https://github.com/yandex/ClickHouse/pull/3687) +* Исправлено поведение функций с состоянием типа `rowNumberInAllBlocks` - раньше они выдавали число на единицу больше вследствие их запуска во время анализа запроса. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3729) +* При невозможности удалить файл `force_restore_data`, выводится сообщение об ошибке. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3794) ### Улучшение сборки: -* Обновлена библиотека `jemalloc`, что исправляет потенциальную утечку памяти. Amos Bird #3557 -* Для debug сборок включено по-умолчанию профилирование `jemalloc`. 2cc82f5cbe266421cd4c1165286c2c47e5ffcb15 -* Добавлена возможность запуска интеграционных тестов, при наличии установленным в системе лишь `Docker`. #3650 -* Добавлен fuzz тест выражений в SELECT запросах. #3442 -* Добавлен покоммитный стресс-тест, выполняющий функциональные тесты параллельно и в произвольном порядке, позволяющий обнаружить больше race conditions. #3438 -* Улучшение способа запуска clickhouse-server в Docker образе. Elghazal Ahmed #3663 -* Для Docker образа добавлена поддержка инициализации базы данных с помощью файлов в директории `/docker-entrypoint-initdb.d`. Konstantin Lebedev #3695 -* Исправления для сборки под ARM. #3709 +* Обновлена библиотека `jemalloc`, что исправляет потенциальную утечку памяти. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3557) +* Для debug сборок включено по-умолчанию профилирование `jemalloc`. [2cc82f5c](https://github.com/yandex/ClickHouse/commit/2cc82f5cbe266421cd4c1165286c2c47e5ffcb15) +* Добавлена возможность запуска интеграционных тестов, при наличии установленным в системе лишь `Docker`. [#3650](https://github.com/yandex/ClickHouse/pull/3650) +* Добавлен fuzz тест выражений в SELECT запросах. [#3442](https://github.com/yandex/ClickHouse/pull/3442) +* Добавлен покоммитный стресс-тест, выполняющий функциональные тесты параллельно и в произвольном порядке, позволяющий обнаружить больше race conditions. [#3438](https://github.com/yandex/ClickHouse/pull/3438) +* Улучшение способа запуска clickhouse-server в Docker образе. [Elghazal Ahmed](https://github.com/yandex/ClickHouse/pull/3663) +* Для Docker образа добавлена поддержка инициализации базы данных с помощью файлов в директории `/docker-entrypoint-initdb.d`. [Konstantin Lebedev](https://github.com/yandex/ClickHouse/pull/3695) +* Исправления для сборки под ARM. [#3709](https://github.com/yandex/ClickHouse/pull/3709) ### Обратно несовместимые изменения: -* Удалена возможность сравнения типа `Date` с числом, необходимо вместо `toDate('2018-12-18') = 17883`, использовать явное приведение типов `= toDate(17883)` #3687 +* Удалена возможность сравнения типа `Date` с числом, необходимо вместо `toDate('2018-12-18') = 17883`, использовать явное приведение типов `= toDate(17883)` [#3687](https://github.com/yandex/ClickHouse/pull/3687) ## ClickHouse release 18.14.18, 2018-12-04 From 88706176e4df332c83172e8fda2f588c8c929434 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 19 Dec 2018 06:35:24 +0300 Subject: [PATCH 068/181] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b9e23d6ea71..00ae12339b2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,7 +3,7 @@ ### Bug fixes: * Fixed error in `dictGet...` function for dictionaries of type `range`, if one of the arguments is constant and other is not. [#3751](https://github.com/yandex/ClickHouse/pull/3751) * Fixed error that caused messages `netlink: '...': attribute type 1 has an invalid length` to be printed in Linux kernel log, that was happening only on fresh enough versions of Linux kernel. [#3749](https://github.com/yandex/ClickHouse/pull/3749) -* Fixed segfault in function `empty` for argument of `FixedString` type. [#3703](https://github.com/yandex/ClickHouse/pull/3703) +* Fixed segfault in function `empty` for argument of `FixedString` type. [Daniel, Dao Quang Minh](https://github.com/yandex/ClickHouse/pull/3703) * Fixed excessive memory allocation when using large value of `max_query_size` setting (a memory chunk of `max_query_size` bytes was preallocated at once). [#3720](https://github.com/yandex/ClickHouse/pull/3720) ### Build changes: From b1bcf82670da2928b563efa7c07530dd752a76f3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 19 Dec 2018 06:37:13 +0300 Subject: [PATCH 069/181] Update CHANGELOG_RU.md --- CHANGELOG_RU.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 86ea3544a19..0f31a6a3ab3 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -5,7 +5,7 @@ * Вычисление `DEFAULT` выражений для отсутствующих полей при загрузке данных в полуструктурированных форматах (`JSONEachRow`, `TSKV`). [#3555](https://github.com/yandex/ClickHouse/pull/3555) * Для запроса `ALTER TABLE` добавлено действие `MODIFY ORDER BY` для изменения ключа сортировки при одновременном добавлении или удалении столбца таблицы. Это полезно для таблиц семейства `MergeTree`, выполняющих дополнительную работу при слияниях, согласно этому ключу сортировки, как например, `SummingMergeTree`, `AggregatingMergeTree` и т. п. [#3581](https://github.com/yandex/ClickHouse/pull/3581) [#3755](https://github.com/yandex/ClickHouse/pull/3755) * Для таблиц семейства `MergeTree` появилась возможность указать различный ключ сортировки (`ORDER BY`) и индекс (`PRIMARY KEY`). Ключ сортировки может быть длиннее, чем индекс. [#3581](https://github.com/yandex/ClickHouse/pull/3581) -* Добавлена табличная функция `hdfs` и движок таблиц `HDFS` для импорта и экспорта данных в HDFS. chenxing-xc [#3617](https://github.com/yandex/ClickHouse/pull/3617) +* Добавлена табличная функция `hdfs` и движок таблиц `HDFS` для импорта и экспорта данных в HDFS. [chenxing-xc](https://github.com/yandex/ClickHouse/pull/3617) * Добавлены функции для работы с base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. [Alexander Krasheninnikov](https://github.com/yandex/ClickHouse/pull/3350) * Для агрегатной функции `uniqCombined` появилась возможность настраивать точность работы с помощью параметра (выбирать количество ячеек HyperLogLog). [#3406](https://github.com/yandex/ClickHouse/pull/3406) * Добавлена таблица `system.contributors`, содержащая имена всех, кто делал коммиты в ClickHouse. [#3452](https://github.com/yandex/ClickHouse/pull/3452) From 8889f0859a47cd6b4fdae40f8441cb66256f9022 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 06:43:37 +0300 Subject: [PATCH 070/181] Fixed typo [#CLICKHOUSE-2] --- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- dbms/src/Interpreters/evaluateConstantExpression.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 29efda0ce63..502bd7198c3 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -269,7 +269,7 @@ private: void assertAggregation() const; /** - * Create Set from a subuqery or a table expression in the query. The created set is suitable for using the index. + * Create Set from a subuquery or a table expression in the query. The created set is suitable for using the index. * The set will not be created if its size hits the limit. */ void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index adb60d34eaa..769f45f9c31 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -57,7 +57,7 @@ std::pair> evaluateConstantExpression(co ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & context) { - /// Branch with string in qery. + /// Branch with string in query. if (typeid_cast(node.get())) return node; From 68e0af176e5819a31527c725c0f3b85cbabc67d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 06:53:09 +0300 Subject: [PATCH 071/181] Fix use after free in arrayEnumerate [#CLICKHOUSE-2] --- dbms/src/Functions/arrayEnumerateExtended.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/arrayEnumerateExtended.h b/dbms/src/Functions/arrayEnumerateExtended.h index 86eba2ded7c..f6e6d455729 100644 --- a/dbms/src/Functions/arrayEnumerateExtended.h +++ b/dbms/src/Functions/arrayEnumerateExtended.h @@ -88,10 +88,11 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C bool has_nullable_columns = false; + ColumnPtr array_holder; for (size_t i = 0; i < arguments.size(); ++i) { - ColumnPtr array_ptr = block.getByPosition(arguments[i]).column; - const ColumnArray * array = checkAndGetColumn(array_ptr.get()); + array_holder = block.getByPosition(arguments[i]).column; + const ColumnArray * array = checkAndGetColumn(array_holder.get()); if (!array) { const ColumnConst * const_array = checkAndGetColumnConst( @@ -100,8 +101,8 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName() + " of " + toString(i + 1) + "-th argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - array_ptr = const_array->convertToFullColumn(); - array = checkAndGetColumn(array_ptr.get()); + array_holder = const_array->convertToFullColumn(); + array = checkAndGetColumn(array_holder.get()); } const ColumnArray::Offsets & offsets_i = array->getOffsets(); From 2874061410825eace8db871f8e805ec4332184fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 07:21:23 +0300 Subject: [PATCH 072/181] Fix use after free in arrayEnumerate [#CLICKHOUSE-2] --- dbms/src/Functions/arrayEnumerateExtended.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/arrayEnumerateExtended.h b/dbms/src/Functions/arrayEnumerateExtended.h index f6e6d455729..159d607d09c 100644 --- a/dbms/src/Functions/arrayEnumerateExtended.h +++ b/dbms/src/Functions/arrayEnumerateExtended.h @@ -88,11 +88,11 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C bool has_nullable_columns = false; - ColumnPtr array_holder; + Columns array_holders; for (size_t i = 0; i < arguments.size(); ++i) { - array_holder = block.getByPosition(arguments[i]).column; - const ColumnArray * array = checkAndGetColumn(array_holder.get()); + const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column; + const ColumnArray * array = checkAndGetColumn(array_ptr.get()); if (!array) { const ColumnConst * const_array = checkAndGetColumnConst( @@ -101,8 +101,8 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName() + " of " + toString(i + 1) + "-th argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - array_holder = const_array->convertToFullColumn(); - array = checkAndGetColumn(array_holder.get()); + array_holders.emplace_back(const_array->convertToFullColumn()); + array = checkAndGetColumn(array_holders.back().get()); } const ColumnArray::Offsets & offsets_i = array->getOffsets(); From 714bd5b88a6e9d0c5897f0963b3b22c2cf6ee0e0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 19 Dec 2018 12:02:59 +0300 Subject: [PATCH 073/181] Add lost zero byte in regexpQuoteMeta --- dbms/src/Functions/regexpQuoteMeta.cpp | 6 ++++-- dbms/src/Functions/trim.cpp | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/regexpQuoteMeta.cpp b/dbms/src/Functions/regexpQuoteMeta.cpp index 2e45a85376c..b14a58ba366 100644 --- a/dbms/src/Functions/regexpQuoteMeta.cpp +++ b/dbms/src/Functions/regexpQuoteMeta.cpp @@ -94,10 +94,12 @@ public: /// suboptimal, but uses original implementation from re2 re2_st::StringPiece unquoted(source, srclen); const auto & quoted = re2_st::RE2::QuoteMeta(unquoted); - std::memcpy(dst_pos, quoted.data(), quoted.size()); + const auto size = quoted.size(); + std::memcpy(dst_pos, quoted.data(), size); source += srclen + 1; - dst_pos += quoted.size() + 1; + dst_pos[size] = '\0'; + dst_pos += size + 1; dst_offsets[row] = dst_pos - dst; src_offset_prev = src_offsets[row]; diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index f6f9d22496a..42aa59bffe5 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -62,7 +62,7 @@ public: res_data.resize(res_data.size() + length + 1); std::memcpy(&res_data[res_offset], start, length); res_offset += length + 1; - res_data[res_offset - 1] = 0; + res_data[res_offset - 1] = '\0'; res_offsets[i] = res_offset; prev_offset = offsets[i]; From 0b79053afda0412bd457a83455d53953b836a2a5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 19 Dec 2018 12:05:04 +0300 Subject: [PATCH 074/181] remove accidental file duplicate --- dbms/src/Functions/right.cpp | 117 ----------------------------------- 1 file changed, 117 deletions(-) delete mode 100644 dbms/src/Functions/right.cpp diff --git a/dbms/src/Functions/right.cpp b/dbms/src/Functions/right.cpp deleted file mode 100644 index 2e45a85376c..00000000000 --- a/dbms/src/Functions/right.cpp +++ /dev/null @@ -1,117 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#if USE_RE2_ST - #include // Y_IGNORE -#else - #define re2_st re2 -#endif - -namespace DB -{ -using namespace GatherUtils; - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -class FunctionRegexpQuoteMeta : public IFunction -{ -public: - static constexpr auto name = "regexpQuoteMeta"; - - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override - { - return 1; - } - - bool useDefaultImplementationForConstants() const override - { - return true; - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - if (!WhichDataType(arguments[0].type).isString()) - throw Exception( - "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override - { - const ColumnPtr column_string = block.getByPosition(arguments[0]).column; - const ColumnString * input = checkAndGetColumn(column_string.get()); - - if (!input) - throw Exception( - "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - - auto dst_column = ColumnString::create(); - auto & dst_data = dst_column->getChars(); - auto & dst_offsets = dst_column->getOffsets(); - - dst_data.resize(input->getChars().size() * input->size()); - dst_offsets.resize(input_rows_count); - - const ColumnString::Offsets & src_offsets = input->getOffsets(); - - auto source = reinterpret_cast(input->getChars().data()); - auto dst = reinterpret_cast(dst_data.data()); - auto dst_pos = dst; - - size_t src_offset_prev = 0; - - for (size_t row = 0; row < input_rows_count; ++row) - { - size_t srclen = src_offsets[row] - src_offset_prev - 1; - - /// suboptimal, but uses original implementation from re2 - re2_st::StringPiece unquoted(source, srclen); - const auto & quoted = re2_st::RE2::QuoteMeta(unquoted); - std::memcpy(dst_pos, quoted.data(), quoted.size()); - - source += srclen + 1; - dst_pos += quoted.size() + 1; - - dst_offsets[row] = dst_pos - dst; - src_offset_prev = src_offsets[row]; - } - - dst_data.resize(dst_pos - dst); - - block.getByPosition(result).column = std::move(dst_column); - } - -}; - -void registerFunctionRegexpQuoteMeta(FunctionFactory & factory) -{ - factory.registerFunction(); -} -} From f88b4975556066c5089a0802f826bee0770e311e Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 19 Dec 2018 13:27:07 +0300 Subject: [PATCH 075/181] Introduce common parser for interval kind names and use it for DATE_DIFF, EXTRACT and INTERVAL (allows to use same set of aliases in them) --- dbms/src/Parsers/CommonParsers.h | 69 +++++++++++ dbms/src/Parsers/ExpressionElementParsers.cpp | 116 ++++++++++-------- dbms/src/Parsers/ExpressionListParsers.cpp | 48 +++++--- .../queries/0_stateless/00619_extract.sql | 2 +- 4 files changed, 167 insertions(+), 68 deletions(-) diff --git a/dbms/src/Parsers/CommonParsers.h b/dbms/src/Parsers/CommonParsers.h index 414f4ceccbc..01944b6d21e 100644 --- a/dbms/src/Parsers/CommonParsers.h +++ b/dbms/src/Parsers/CommonParsers.h @@ -46,4 +46,73 @@ protected: } }; +class ParserInterval: public IParserBase +{ +public: + enum class IntervalKind + { + Incorrect, + Second, + Minute, + Hour, + Day, + Week, + Month, + Quarter, + Year + }; + + IntervalKind interval_kind; + + ParserInterval() : interval_kind(IntervalKind::Incorrect) {} + +protected: + const char * getName() const override { return "interval"; } + + bool parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected) override + { + if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected) + || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) + interval_kind = IntervalKind::Second; + else if ( + ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected) + || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) + interval_kind = IntervalKind::Minute; + else if ( + ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected) + || ParserKeyword("HH").ignore(pos, expected)) + interval_kind = IntervalKind::Hour; + else if ( + ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected) + || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) + interval_kind = IntervalKind::Day; + else if ( + ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected) + || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) + interval_kind = IntervalKind::Week; + else if ( + ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected) + || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) + interval_kind = IntervalKind::Month; + else if ( + ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected) + || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) + interval_kind = IntervalKind::Quarter; + else if ( + ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected) + || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) + interval_kind = IntervalKind::Year; + else + interval_kind = IntervalKind::Incorrect; + + if (interval_kind == IntervalKind::Incorrect) + { + expected.add(pos, "YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE or SECOND"); + return false; + } + /// one of ParserKeyword already made ++pos + return true; + } +}; + } diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 6de29f097d8..ba7c7dbf108 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -651,26 +651,42 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp ASTPtr expr; const char * function_name = nullptr; - if (ParserKeyword("SECOND").ignore(pos, expected)) - function_name = "toSecond"; - else if (ParserKeyword("MINUTE").ignore(pos, expected)) - function_name = "toMinute"; - else if (ParserKeyword("HOUR").ignore(pos, expected)) - function_name = "toHour"; - else if (ParserKeyword("DAY").ignore(pos, expected)) - function_name = "toDayOfMonth"; - - // TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15'))) - // else if (ParserKeyword("WEEK").ignore(pos, expected)) - // function_name = "toRelativeWeekNum"; - - else if (ParserKeyword("MONTH").ignore(pos, expected)) - function_name = "toMonth"; - else if (ParserKeyword("YEAR").ignore(pos, expected)) - function_name = "toYear"; - else + ParserInterval interval_parser; + if (!interval_parser.ignore(pos, expected)) return false; + switch (interval_parser.interval_kind) + { + case ParserInterval::IntervalKind::Second: + function_name = "toSecond"; + break; + case ParserInterval::IntervalKind::Minute: + function_name = "toMinute"; + break; + case ParserInterval::IntervalKind::Hour: + function_name = "toHour"; + break; + case ParserInterval::IntervalKind::Day: + function_name = "toDayOfMonth"; + break; + case ParserInterval::IntervalKind::Week: + // TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15'))) + // else if (ParserKeyword("WEEK").ignore(pos, expected)) + // function_name = "toRelativeWeekNum"; + return false; + case ParserInterval::IntervalKind::Month: + function_name = "toMonth"; + break; + case ParserInterval::IntervalKind::Quarter: + function_name = "toQuarter"; + break; + case ParserInterval::IntervalKind::Year: + function_name = "toYear"; + break; + default: + return false; + } + ParserKeyword s_from("FROM"); if (!s_from.ignore(pos, expected)) return false; @@ -711,40 +727,40 @@ bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return false; ++pos; - if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected) - || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) - interval_name = "second"; - else if ( - ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected) - || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) - interval_name = "minute"; - else if ( - ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected) - || ParserKeyword("HH").ignore(pos, expected)) - interval_name = "hour"; - else if ( - ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected) - || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) - interval_name = "day"; - else if ( - ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected) - || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) - interval_name = "week"; - else if ( - ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected) - || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) - interval_name = "month"; - else if ( - ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected) - || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) - interval_name = "quarter"; - else if ( - ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected) - || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) - interval_name = "year"; - else + ParserInterval interval_parser; + if (!interval_parser.ignore(pos, expected)) return false; + switch (interval_parser.interval_kind) + { + case ParserInterval::IntervalKind::Second: + interval_name = "second"; + break; + case ParserInterval::IntervalKind::Minute: + interval_name = "minute"; + break; + case ParserInterval::IntervalKind::Hour: + interval_name = "hour"; + break; + case ParserInterval::IntervalKind::Day: + interval_name = "day"; + break; + case ParserInterval::IntervalKind::Week: + interval_name = "week"; + break; + case ParserInterval::IntervalKind::Month: + interval_name = "month"; + break; + case ParserInterval::IntervalKind::Quarter: + interval_name = "quarter"; + break; + case ParserInterval::IntervalKind::Year: + interval_name = "year"; + break; + default: + return false; + } + if (pos->type != TokenType::Comma) return false; ++pos; diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index 9365606f5dd..75b20226fb6 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -609,25 +609,39 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec const char * function_name = nullptr; - if (ParserKeyword("SECOND").ignore(pos, expected)) - function_name = "toIntervalSecond"; - else if (ParserKeyword("MINUTE").ignore(pos, expected)) - function_name = "toIntervalMinute"; - else if (ParserKeyword("HOUR").ignore(pos, expected)) - function_name = "toIntervalHour"; - else if (ParserKeyword("DAY").ignore(pos, expected)) - function_name = "toIntervalDay"; - else if (ParserKeyword("WEEK").ignore(pos, expected)) - function_name = "toIntervalWeek"; - else if (ParserKeyword("MONTH").ignore(pos, expected)) - function_name = "toIntervalMonth"; - else if (ParserKeyword("QUARTER").ignore(pos, expected)) - function_name = "toIntervalQuarter"; - else if (ParserKeyword("YEAR").ignore(pos, expected)) - function_name = "toIntervalYear"; - else + ParserInterval interval_parser; + if (!interval_parser.ignore(pos, expected)) return false; + switch (interval_parser.interval_kind) + { + case ParserInterval::IntervalKind::Second: + function_name = "toIntervalSecond"; + break; + case ParserInterval::IntervalKind::Minute: + function_name = "toIntervalMinute"; + break; + case ParserInterval::IntervalKind::Hour: + function_name = "toIntervalHour"; + break; + case ParserInterval::IntervalKind::Day: + function_name = "toIntervalDay"; + break; + case ParserInterval::IntervalKind::Week: + function_name = "toIntervalWeek"; + break; + case ParserInterval::IntervalKind::Month: + function_name = "toIntervalMonth"; + break; + case ParserInterval::IntervalKind::Quarter: + function_name = "toIntervalQuarter"; + break; + case ParserInterval::IntervalKind::Year: + function_name = "toIntervalYear"; + break; + default: + return false; + } /// the function corresponding to the operator auto function = std::make_shared(); diff --git a/dbms/tests/queries/0_stateless/00619_extract.sql b/dbms/tests/queries/0_stateless/00619_extract.sql index 78ec812dad6..034ae55b5e3 100644 --- a/dbms/tests/queries/0_stateless/00619_extract.sql +++ b/dbms/tests/queries/0_stateless/00619_extract.sql @@ -13,7 +13,7 @@ SELECT EXTRACT(year FROM toDateTime('2017-12-31 18:59:58')); DROP TABLE IF EXISTS test.Orders; CREATE TABLE test.Orders (OrderId UInt64, OrderName String, OrderDate DateTime) engine = Log; insert into test.Orders values (1, 'Jarlsberg Cheese', toDateTime('2008-10-11 13:23:44')); -SELECT EXTRACT(YEAR FROM OrderDate) AS OrderYear, EXTRACT(MONTH FROM OrderDate) AS OrderMonth, EXTRACT(DAY FROM OrderDate) AS OrderDay, +SELECT EXTRACT(YYYY FROM OrderDate) AS OrderYear, EXTRACT(MONTH FROM OrderDate) AS OrderMonth, EXTRACT(DAY FROM OrderDate) AS OrderDay, EXTRACT(HOUR FROM OrderDate), EXTRACT(MINUTE FROM OrderDate), EXTRACT(SECOND FROM OrderDate) FROM test.Orders WHERE OrderId=1; DROP TABLE test.Orders; From 0352c5a57c8a3124f60dad8506b03fa3b41bd3e0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 19 Dec 2018 14:37:14 +0300 Subject: [PATCH 076/181] Introduce DATE_ADD/DATE_SUB/TIMESTAMP_ADD/TIMESTAMP_SUB #3705 --- dbms/src/Parsers/CommonParsers.h | 25 +++++++ dbms/src/Parsers/ExpressionElementParsers.cpp | 68 ++++++++++++++++++- dbms/src/Parsers/ExpressionElementParsers.h | 7 ++ dbms/src/Parsers/ExpressionListParsers.cpp | 32 +-------- .../00765_sql_compatibility_aliases.reference | 2 + .../00765_sql_compatibility_aliases.sql | 2 + 6 files changed, 105 insertions(+), 31 deletions(-) diff --git a/dbms/src/Parsers/CommonParsers.h b/dbms/src/Parsers/CommonParsers.h index 01944b6d21e..44c8ab17fb7 100644 --- a/dbms/src/Parsers/CommonParsers.h +++ b/dbms/src/Parsers/CommonParsers.h @@ -66,6 +66,31 @@ public: ParserInterval() : interval_kind(IntervalKind::Incorrect) {} + const char * getToIntervalKindFunctionName() + { + switch (interval_kind) + { + case ParserInterval::IntervalKind::Second: + return "toIntervalSecond"; + case ParserInterval::IntervalKind::Minute: + return "toIntervalMinute"; + case ParserInterval::IntervalKind::Hour: + return "toIntervalHour"; + case ParserInterval::IntervalKind::Day: + return "toIntervalDay"; + case ParserInterval::IntervalKind::Week: + return "toIntervalWeek"; + case ParserInterval::IntervalKind::Month: + return "toIntervalMonth"; + case ParserInterval::IntervalKind::Quarter: + return "toIntervalQuarter"; + case ParserInterval::IntervalKind::Year: + return "toIntervalYear"; + default: + return nullptr; + } + } + protected: const char * getName() const override { return "interval"; } diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index ba7c7dbf108..35e66da5141 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -714,13 +714,78 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return true; } +bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + const char * function_name = nullptr; + ASTPtr left_node; + ASTPtr right_node; + + if (ParserKeyword("DATEADD").ignore(pos, expected) || ParserKeyword("DATE_ADD").ignore(pos, expected) + || ParserKeyword("TIMESTAMPADD").ignore(pos, expected) || ParserKeyword("TIMESTAMP_ADD").ignore(pos, expected)) + function_name = "plus"; + else if (ParserKeyword("DATESUB").ignore(pos, expected) || ParserKeyword("DATE_SUB").ignore(pos, expected) + || ParserKeyword("TIMESTAMPSUB").ignore(pos, expected) || ParserKeyword("TIMESTAMP_SUB").ignore(pos, expected)) + function_name = "minus"; + else + return false; + + if (pos->type != TokenType::OpeningRoundBracket) + return false; + ++pos; + + ParserInterval interval_parser; + if (!interval_parser.ignore(pos, expected)) + return false; + + const char * interval_function_name = interval_parser.getToIntervalKindFunctionName(); + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, left_node, expected)) + return false; + + if (pos->type != TokenType::Comma) + return false; + ++pos; + + if (!ParserExpression().parse(pos, right_node, expected)) + return false; + + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + + auto interval_expr_list_args = std::make_shared(); + interval_expr_list_args->children = {left_node}; + + auto interval_func_node = std::make_shared(); + interval_func_node->name = interval_function_name; + interval_func_node->arguments = std::move(interval_expr_list_args); + interval_func_node->children.push_back(interval_func_node->arguments); + + auto expr_list_args = std::make_shared(); + expr_list_args->children = {right_node, interval_func_node}; + + auto func_node = std::make_shared(); + func_node->name = function_name; + func_node->arguments = std::move(expr_list_args); + func_node->children.push_back(func_node->arguments); + + node = std::move(func_node); + + return true; +} + bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { const char * interval_name = nullptr; ASTPtr left_node; ASTPtr right_node; - if (!(ParserKeyword("DATEDIFF").ignore(pos, expected) || ParserKeyword("DATE_DIFF").ignore(pos, expected))) + if (!(ParserKeyword("DATEDIFF").ignore(pos, expected) || ParserKeyword("DATE_DIFF").ignore(pos, expected) + || ParserKeyword("TIMESTAMPDIFF").ignore(pos, expected) || ParserKeyword("TIMESTAMP_DIFF").ignore(pos, expected))) return false; if (pos->type != TokenType::OpeningRoundBracket) @@ -1093,6 +1158,7 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserLiteral().parse(pos, node, expected) || ParserCastExpression().parse(pos, node, expected) || ParserExtractExpression().parse(pos, node, expected) + || ParserDateAddExpression().parse(pos, node, expected) || ParserDateDiffExpression().parse(pos, node, expected) || ParserSubstringExpression().parse(pos, node, expected) || ParserTrimExpression().parse(pos, node, expected) diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index c35a6613155..c6afbd171e4 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -131,6 +131,13 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserDateAddExpression : public IParserBase +{ +protected: + const char * getName() const override { return "DATE_ADD expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + class ParserDateDiffExpression : public IParserBase { protected: diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index 75b20226fb6..de6fc2dc129 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -607,41 +607,13 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec if (!ParserExpressionWithOptionalAlias(false).parse(pos, expr, expected)) return false; - const char * function_name = nullptr; ParserInterval interval_parser; if (!interval_parser.ignore(pos, expected)) return false; - switch (interval_parser.interval_kind) - { - case ParserInterval::IntervalKind::Second: - function_name = "toIntervalSecond"; - break; - case ParserInterval::IntervalKind::Minute: - function_name = "toIntervalMinute"; - break; - case ParserInterval::IntervalKind::Hour: - function_name = "toIntervalHour"; - break; - case ParserInterval::IntervalKind::Day: - function_name = "toIntervalDay"; - break; - case ParserInterval::IntervalKind::Week: - function_name = "toIntervalWeek"; - break; - case ParserInterval::IntervalKind::Month: - function_name = "toIntervalMonth"; - break; - case ParserInterval::IntervalKind::Quarter: - function_name = "toIntervalQuarter"; - break; - case ParserInterval::IntervalKind::Year: - function_name = "toIntervalYear"; - break; - default: - return false; - } + const char * function_name = interval_parser.getToIntervalKindFunctionName(); + /// the function corresponding to the operator auto function = std::make_shared(); diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index 4e05015901b..43e7758e0f1 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -23,3 +23,5 @@ fooabbafoo foo* -11 -3 +2021-01-01 +2018-07-18 01:02:03 diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index 6fea2dafe1c..6373d579f39 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -25,3 +25,5 @@ select TRIM(both 'ab' FROM 'abbafooabbafooabba'); select trim(LEADING '*[]{}|\\' FROM '\\|[[[}}}*foo*'); select DATE_DIFF(MONTH, toDate('2018-12-18'), toDate('2018-01-01')); select DATE_DIFF(QQ, toDate('2018-12-18'), toDate('2018-01-01')); +select DATE_ADD(YEAR, 3, toDate('2018-01-01')); +select timestamp_sub(SQL_TSI_MONTH, 5, toDateTime('2018-12-18 01:02:03')); From 52eb11a3100c69b4569faec2eb6ff643755fcf39 Mon Sep 17 00:00:00 2001 From: Ilya Skrypitsa Date: Wed, 19 Dec 2018 14:38:52 +0300 Subject: [PATCH 077/181] Typo in Creating a Table example (#3872) --- docs/en/operations/table_engines/summingmergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/table_engines/summingmergetree.md b/docs/en/operations/table_engines/summingmergetree.md index 587613c33d7..286507d8b9c 100644 --- a/docs/en/operations/table_engines/summingmergetree.md +++ b/docs/en/operations/table_engines/summingmergetree.md @@ -13,7 +13,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) ENGINE = MergeTree() +) ENGINE = SummingMergeTree() [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] From b9440fe99db35627a1f6a8266f4da1bc30bbc11f Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 19 Dec 2018 15:38:13 +0300 Subject: [PATCH 078/181] Another minor refactoring. --- .../evaluateConstantExpression.cpp | 8 +- .../Interpreters/evaluateConstantExpression.h | 16 ++-- dbms/src/Storages/StorageDistributed.cpp | 90 ++++++++++++++++--- .../StorageDistributedShardsOptimizer.cpp | 82 ----------------- ...ize_skip_select_on_unused_shards.reference | 1 + ...d_optimize_skip_select_on_unused_shards.sh | 9 +- 6 files changed, 94 insertions(+), 112 deletions(-) delete mode 100644 dbms/src/Storages/StorageDistributedShardsOptimizer.cpp diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index 8d422c248b1..29753a4c637 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -243,9 +243,7 @@ namespace } } -// TODO: distinguish always-false and failed evaluation results, -// assume failed if returned empty `Blocks` for now. -Blocks evaluateConstantExpressionAsBlock(const ASTPtr & node, const ExpressionActionsPtr & target_expr) +std::optional evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr) { Blocks result; @@ -257,7 +255,7 @@ Blocks evaluateConstantExpressionAsBlock(const ASTPtr & node, const ExpressionAc if (dnf.empty()) { - return result; + return {}; } auto hasRequiredColumns = [&target_expr](const Block & block) -> bool @@ -310,7 +308,7 @@ Blocks evaluateConstantExpressionAsBlock(const ASTPtr & node, const ExpressionAc } } - return result; + return {result}; } } diff --git a/dbms/src/Interpreters/evaluateConstantExpression.h b/dbms/src/Interpreters/evaluateConstantExpression.h index e13a816149a..53175b1e6b9 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.h +++ b/dbms/src/Interpreters/evaluateConstantExpression.h @@ -23,20 +23,24 @@ using ExpressionActionsPtr = std::shared_ptr; std::pair> evaluateConstantExpression(const ASTPtr & node, const Context & context); -/** Evaluate constant expression - * and returns ASTLiteral with its value. +/** Evaluate constant expression and returns ASTLiteral with its value. */ ASTPtr evaluateConstantExpressionAsLiteral(const ASTPtr & node, const Context & context); -/** Evaluate constant expression - * and returns ASTLiteral with its value. +/** Evaluate constant expression and returns ASTLiteral with its value. * Also, if AST is identifier, then return string literal with its name. * Useful in places where some name may be specified as identifier, or as result of a constant expression. */ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, const Context & context); -// FIXME: collapse returned blocks into a single block. -Blocks evaluateConstantExpressionAsBlock(const ASTPtr & node, const ExpressionActionsPtr & target_expr); +/** Try to fold condition to countable set of constant values. + * @param condition a condition that we try to fold. + * @param target_expr expression evaluated over a set of constants. + * @return optional blocks each with a single row and a single column for target expression, + * or empty blocks if condition is always false, + * or nothing if condition can't be folded to a set of constants. + */ +std::optional evaluateExpressionOverConstantCondition(const ASTPtr & condition, const ExpressionActionsPtr & target_expr); } diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index ad4e7fbe7da..0429db5cef6 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -1,38 +1,41 @@ +#include + #include #include #include #include +#include -#include -#include #include +#include #include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include +#include #include +#include +#include +#include +#include +#include +#include +#include -#include +#include +#include +#include #include #include +#include #include -#include +#include #include -#include -#include #include #include @@ -58,6 +61,7 @@ namespace ErrorCodes extern const int INFINITE_LOOP; extern const int TYPE_MISMATCH; extern const int NO_SUCH_COLUMN_IN_TABLE; + extern const int TOO_MANY_ROWS; } @@ -133,6 +137,29 @@ void initializeFileNamesIncrement(const std::string & path, SimpleIncrement & in increment.set(getMaximumFileNumber(path)); } +/// the same as DistributedBlockOutputStream::createSelector, should it be static? +IColumn::Selector createSelector(const ClusterPtr cluster, const ColumnWithTypeAndName & result) +{ + const auto & slot_to_shard = cluster->getSlotToShard(); + +#define CREATE_FOR_TYPE(TYPE) \ + if (typeid_cast(result.type.get())) \ + return createBlockSelector(*result.column, slot_to_shard); + + CREATE_FOR_TYPE(UInt8) + CREATE_FOR_TYPE(UInt16) + CREATE_FOR_TYPE(UInt32) + CREATE_FOR_TYPE(UInt64) + CREATE_FOR_TYPE(Int8) + CREATE_FOR_TYPE(Int16) + CREATE_FOR_TYPE(Int32) + CREATE_FOR_TYPE(Int64) + +#undef CREATE_FOR_TYPE + + throw Exception{"Sharding key expression does not evaluate to an integer type", ErrorCodes::TYPE_MISMATCH}; +} + } @@ -433,6 +460,41 @@ void StorageDistributed::ClusterNodeData::shutdownAndDropAllData() directory_monitor->shutdownAndDropAllData(); } +/// Returns a new cluster with fewer shards if constant folding for `sharding_key_expr` is possible +/// using constraints from "WHERE" condition, otherwise returns `nullptr` +ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info) +{ + const auto & select = typeid_cast(*query_info.query); + + if (!select.where_expression) + { + return nullptr; + } + + const auto & blocks = evaluateExpressionOverConstantCondition(select.where_expression, sharding_key_expr); + + // Can't get definite answer if we can skip any shards + if (!blocks) + { + return nullptr; + } + + std::set shards; + + for (const auto & block : *blocks) + { + if (!block.has(sharding_key_column_name)) + throw Exception("sharding_key_expr should evaluate as a single row", ErrorCodes::TOO_MANY_ROWS); + + const auto result = block.getByName(sharding_key_column_name); + const auto selector = createSelector(cluster, result); + + shards.insert(selector.begin(), selector.end()); + } + + return cluster->getClusterWithMultipleShards({shards.begin(), shards.end()}); +} + void registerStorageDistributed(StorageFactory & factory) { diff --git a/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp b/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp deleted file mode 100644 index ccdcfea2faa..00000000000 --- a/dbms/src/Storages/StorageDistributedShardsOptimizer.cpp +++ /dev/null @@ -1,82 +0,0 @@ -#include - -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - -extern const int TYPE_MISMATCH; - -} - -namespace -{ - -/// the same as DistributedBlockOutputStream::createSelector, should it be static? -IColumn::Selector createSelector(const ClusterPtr cluster, const ColumnWithTypeAndName & result) -{ - const auto & slot_to_shard = cluster->getSlotToShard(); - -#define CREATE_FOR_TYPE(TYPE) \ - if (typeid_cast(result.type.get())) \ - return createBlockSelector(*result.column, slot_to_shard); - - CREATE_FOR_TYPE(UInt8) - CREATE_FOR_TYPE(UInt16) - CREATE_FOR_TYPE(UInt32) - CREATE_FOR_TYPE(UInt64) - CREATE_FOR_TYPE(Int8) - CREATE_FOR_TYPE(Int16) - CREATE_FOR_TYPE(Int32) - CREATE_FOR_TYPE(Int64) - -#undef CREATE_FOR_TYPE - - throw Exception{"Sharding key expression does not evaluate to an integer type", ErrorCodes::TYPE_MISMATCH}; -} - -} - -/// Returns a new cluster with fewer shards if constant folding for `sharding_key_expr` is possible -/// using constraints from "WHERE" condition, otherwise returns `nullptr` -ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info) -{ - const auto & select = typeid_cast(*query_info.query); - - if (!select.where_expression) - { - return nullptr; - } - - const auto blocks = evaluateConstantExpressionAsBlock(select.where_expression, sharding_key_expr); - - // Can't get definite answer if we can skip any shards - if (blocks.empty()) - { - return nullptr; - } - - std::set shards; - - for (const auto & block : blocks) - { - if (!block.has(sharding_key_column_name)) - throw Exception("sharding_key_expr should evaluate as a single row", ErrorCodes::TYPE_MISMATCH); - - const auto result = block.getByName(sharding_key_column_name); - const auto selector = createSelector(cluster, result); - - shards.insert(selector.begin(), selector.end()); - } - - return cluster->getClusterWithMultipleShards({shards.begin(), shards.end()}); -} - -} diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference index 3161be13b12..add8c239ade 100644 --- a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference +++ b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.reference @@ -2,6 +2,7 @@ OK OK 1 OK +0 4 2 1 diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh index 84fec6de872..92af8677058 100755 --- a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh +++ b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh @@ -36,11 +36,10 @@ ${CLICKHOUSE_CLIENT} -n --query=" # Try more complext expressions for constant folding - all should pass. -# TODO: should pass one day. -#${CLICKHOUSE_CLIENT} -n --query=" -# SET distributed_optimize_skip_select_on_unused_shards = 1; -# SELECT count(*) FROM test.distributed WHERE a = 1 AND a = 0; -#" +${CLICKHOUSE_CLIENT} -n --query=" + SET distributed_optimize_skip_select_on_unused_shards = 1; + SELECT count(*) FROM test.distributed WHERE a = 1 AND a = 0 AND b = 0; +" ${CLICKHOUSE_CLIENT} -n --query=" SET distributed_optimize_skip_select_on_unused_shards = 1; From 1ae355a8e431a0aea87ebe99de8965671133e2f5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 15:56:58 +0300 Subject: [PATCH 079/181] Fixed error in changelog [#CLICKHOUSE-2] --- CHANGELOG_RU.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 0f31a6a3ab3..a03d064cf4b 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -31,7 +31,7 @@ * Исправлена работа некоторых случаев `VIEW` и подзапросов без указания базы данных. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/3521) * Исправлен race condition при одновременном чтении из `MATERIALIZED VIEW` и удалением `MATERIALIZED VIEW` из-за отсутствия блокировки внутренней таблицы `MATERIALIZED VIEW`. [#3404](https://github.com/yandex/ClickHouse/pull/3404) [#3694](https://github.com/yandex/ClickHouse/pull/3694) * Исправлена ошибка `Lock handler cannot be nullptr.` [#3689](https://github.com/yandex/ClickHouse/pull/3689) -* Исправления выполнения запросов при включенной настройке `compile_expressions` (выключена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. [#3457](https://github.com/yandex/ClickHouse/pull/3457) +* Исправления выполнения запросов при включенной настройке `compile_expressions` (включена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. [#3457](https://github.com/yandex/ClickHouse/pull/3457) * Исправлено падение при указании неконстантного аргумента scale в функциях `toDecimal32/64/128`. * Исправлена ошибка при попытке вставки в формате `Values` массива с `NULL` элементами в столбец типа `Array` без `Nullable` (в случае `input_format_values_interpret_expressions` = 1). [#3487](https://github.com/yandex/ClickHouse/pull/3487) [#3503](https://github.com/yandex/ClickHouse/pull/3503) * Исправлено непрерывное логгирование ошибок в `DDLWorker`, если ZooKeeper недоступен. [8f50c620](https://github.com/yandex/ClickHouse/commit/8f50c620334988b28018213ec0092fe6423847e2) From b7ce79c423fc4b1f668b4566ed77ee9d35397b0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 16:11:06 +0300 Subject: [PATCH 080/181] Miscellaneous [#CLICKHOUSE-2] --- dbms/src/Common/Config/ConfigProcessor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 081bcd7c995..7e9579366b1 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -24,7 +24,7 @@ namespace DB { /// For cutting prerpocessed path to this base -std::string main_config_path; +static std::string main_config_path; /// Extracts from a string the first encountered number consisting of at least two digits. static std::string numberFromHost(const std::string & s) From cbf3c34cf3eeb6222c7201bd88a410a982a88f3d Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 19 Dec 2018 16:13:51 +0300 Subject: [PATCH 081/181] minor improvement in ExpressionAnalyzer + array join --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 28 +++++++++++--------- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 8 ++++-- dbms/src/Parsers/ASTSelectQuery.cpp | 12 ++++----- dbms/src/Parsers/ASTSelectQuery.h | 2 +- 5 files changed, 29 insertions(+), 23 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index dda46a008ad..e488aa5d81b 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -143,15 +143,17 @@ void ExpressionAnalyzer::analyzeAggregation() ExpressionActionsPtr temp_actions = std::make_shared(source_columns, context); - if (select_query && select_query->array_join_expression_list()) - { - getRootActions(select_query->array_join_expression_list(), true, temp_actions); - addMultipleArrayJoinAction(temp_actions); - array_join_columns = temp_actions->getSampleBlock().getNamesAndTypesList(); - } - if (select_query) { + bool is_array_join_left; + ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left); + if (array_join_expression_list) + { + getRootActions(array_join_expression_list, true, temp_actions); + addMultipleArrayJoinAction(temp_actions, is_array_join_left); + array_join_columns = temp_actions->getSampleBlock().getNamesAndTypesList(); + } + const ASTTablesInSelectQueryElement * join = select_query->join(); if (join) { @@ -512,7 +514,7 @@ void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAn } /// "Big" ARRAY JOIN. -void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const +void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool array_join_is_left) const { NameSet result_columns; for (const auto & result_source : syntax->array_join_result_to_source) @@ -525,22 +527,24 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio result_columns.insert(result_source.first); } - actions->add(ExpressionAction::arrayJoin(result_columns, select_query->array_join_is_left(), context)); + actions->add(ExpressionAction::arrayJoin(result_columns, array_join_is_left, context)); } bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types) { assertSelect(); - if (!select_query->array_join_expression_list()) + bool is_array_join_left; + ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left); + if (!array_join_expression_list) return false; initChain(chain, source_columns); ExpressionActionsChain::Step & step = chain.steps.back(); - getRootActions(select_query->array_join_expression_list(), only_types, step.actions); + getRootActions(array_join_expression_list, only_types, step.actions); - addMultipleArrayJoinAction(step.actions); + addMultipleArrayJoinAction(step.actions, is_array_join_left); return true; } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 502bd7198c3..ba1e3a252d0 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -240,7 +240,7 @@ private: /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(); - void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const; + void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const; void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 78c90a0be8c..3d9a7f55df3 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -676,9 +676,13 @@ void optimizeUsing(const ASTSelectQuery * select_query) void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, const Names & source_columns, const NameSet & source_columns_set) { - if (select_query && select_query->array_join_expression_list()) + if (!select_query) + return; + + ASTPtr array_join_expression_list = select_query->array_join_expression_list(); + if (array_join_expression_list) { - ASTs & array_join_asts = select_query->array_join_expression_list()->children; + ASTs & array_join_asts = array_join_expression_list->children; for (const auto & ast : array_join_asts) { const String nested_table_name = ast->getColumnName(); diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index 35e9ed75a4e..4a42585b3c1 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -283,23 +283,21 @@ bool ASTSelectQuery::final() const } -ASTPtr ASTSelectQuery::array_join_expression_list() const +ASTPtr ASTSelectQuery::array_join_expression_list(bool & is_left) const { const ASTArrayJoin * array_join = getFirstArrayJoin(*this); if (!array_join) return {}; + is_left = (array_join->kind == ASTArrayJoin::Kind::Left); return array_join->expression_list; } -bool ASTSelectQuery::array_join_is_left() const +ASTPtr ASTSelectQuery::array_join_expression_list() const { - const ASTArrayJoin * array_join = getFirstArrayJoin(*this); - if (!array_join) - return {}; - - return array_join->kind == ASTArrayJoin::Kind::Left; + bool is_left; + return array_join_expression_list(is_left); } diff --git a/dbms/src/Parsers/ASTSelectQuery.h b/dbms/src/Parsers/ASTSelectQuery.h index d9bb3f11be4..bf16e4133de 100644 --- a/dbms/src/Parsers/ASTSelectQuery.h +++ b/dbms/src/Parsers/ASTSelectQuery.h @@ -41,9 +41,9 @@ public: /// Compatibility with old parser of tables list. TODO remove ASTPtr sample_size() const; ASTPtr sample_offset() const; + ASTPtr array_join_expression_list(bool & is_left) const; ASTPtr array_join_expression_list() const; const ASTTablesInSelectQueryElement * join() const; - bool array_join_is_left() const; bool final() const; void replaceDatabaseAndTable(const String & database_name, const String & table_name); void addTableFunction(ASTPtr & table_function_ptr); From ea27ec463e047a0b7e64c6959177a28779d00baa Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Wed, 19 Dec 2018 00:18:54 +0300 Subject: [PATCH 082/181] add SettingLogsLevel --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Interpreters/SettingsCommon.cpp | 55 ++++++++++++++++++++++++ dbms/src/Interpreters/SettingsCommon.h | 21 +++++++++ 4 files changed, 78 insertions(+), 1 deletion(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index e5b6028594b..0b7d089361c 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -402,6 +402,7 @@ namespace ErrorCodes extern const int SYSTEM_ERROR = 425; extern const int NULL_POINTER_DEREFERENCE = 426; extern const int CANNOT_COMPILE_REGEXP = 427; + extern const int UNKNOWN_LOG_LEVEL = 428; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index d5775dd3945..fc8ea2c4630 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -277,7 +277,7 @@ struct Settings M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \ - M(SettingString, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ + M(SettingLogsLevel, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ M(SettingBool, enable_optimize_predicate_expression, 0, "If it is set to true, optimize predicates to subqueries.") \ \ M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \ diff --git a/dbms/src/Interpreters/SettingsCommon.cpp b/dbms/src/Interpreters/SettingsCommon.cpp index 08e5d1b1781..b65097a0f42 100644 --- a/dbms/src/Interpreters/SettingsCommon.cpp +++ b/dbms/src/Interpreters/SettingsCommon.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE; extern const int UNKNOWN_GLOBAL_SUBQUERIES_METHOD; extern const int UNKNOWN_JOIN_STRICTNESS; + extern const int UNKNOWN_LOG_LEVEL; extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; extern const int BAD_ARGUMENTS; } @@ -674,4 +675,58 @@ void SettingDateTimeInputFormat::write(WriteBuffer & buf) const writeBinary(toString(), buf); } + +const std::vector SettingLogsLevel::log_levels = +{ + "none", + "trace", + "debug", + "information", + "warning", + "error" +}; + + +SettingLogsLevel::SettingLogsLevel(const String & level) +{ + set(level); +} + + +void SettingLogsLevel::set(const String & level) +{ + auto it = std::find(log_levels.begin(), log_levels.end(), level); + if (it == log_levels.end()) + throw Exception("Log level '" + level + "' not allowed.", ErrorCodes::UNKNOWN_LOG_LEVEL); + + value = *it; + changed = true; +} + + +void SettingLogsLevel::set(const Field & level) +{ + set(safeGet(level)); +} + + +void SettingLogsLevel::set(ReadBuffer & buf) +{ + String x; + readBinary(x, buf); + set(x); +} + + +String SettingLogsLevel::toString() const +{ + return value; +} + + +void SettingLogsLevel::write(WriteBuffer & buf) const +{ + writeBinary(toString(), buf); +} + } diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index 667912d01be..bfc0f30f8e5 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -404,4 +404,25 @@ struct SettingDateTimeInputFormat void write(WriteBuffer & buf) const; }; + +class SettingLogsLevel +{ +public: + + String value; + bool changed = false; + static const std::vector log_levels; + + SettingLogsLevel(const String & level); + operator String() const { return value; } + void set(const String & level); + void set(const Field & level); + void set(ReadBuffer & buf); + + String toString() const; + void write(WriteBuffer & buf) const; +}; + + + } From 37345468b31e2d2a5463e4480914b85b7f289bb0 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Wed, 19 Dec 2018 16:57:55 +0300 Subject: [PATCH 083/181] Explicitly include for clang-6.0 --- dbms/src/Interpreters/evaluateConstantExpression.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/evaluateConstantExpression.h b/dbms/src/Interpreters/evaluateConstantExpression.h index 53175b1e6b9..a901612040b 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.h +++ b/dbms/src/Interpreters/evaluateConstantExpression.h @@ -1,11 +1,13 @@ #pragma once -#include #include #include #include #include +#include +#include + namespace DB { From 937c0fb4a47bfb62f5723a19b47fd131f2996a33 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 19 Dec 2018 17:02:26 +0300 Subject: [PATCH 084/181] Treat Enum* types as Int* inside If() function. --- dbms/src/Core/callOnTypeIndex.h | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/dbms/src/Core/callOnTypeIndex.h b/dbms/src/Core/callOnTypeIndex.h index 8f4424ec016..ad2a98d8112 100644 --- a/dbms/src/Core/callOnTypeIndex.h +++ b/dbms/src/Core/callOnTypeIndex.h @@ -27,7 +27,6 @@ bool callOnBasicType(TypeIndex number, F && f) case TypeIndex::UInt16: return f(TypePair()); case TypeIndex::UInt32: return f(TypePair()); case TypeIndex::UInt64: return f(TypePair()); - //case TypeIndex::UInt128>: return f(TypePair()); case TypeIndex::Int8: return f(TypePair()); case TypeIndex::Int16: return f(TypePair()); @@ -35,6 +34,9 @@ bool callOnBasicType(TypeIndex number, F && f) case TypeIndex::Int64: return f(TypePair()); case TypeIndex::Int128: return f(TypePair()); + case TypeIndex::Enum8: return f(TypePair()); + case TypeIndex::Enum16: return f(TypePair()); + default: break; } @@ -89,13 +91,16 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f) case TypeIndex::UInt16: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::UInt32: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::UInt64: return callOnBasicType(type_num2, std::forward(f)); - //case TypeIndex::UInt128: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::Int8: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::Int16: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::Int32: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::Int64: return callOnBasicType(type_num2, std::forward(f)); case TypeIndex::Int128: return callOnBasicType(type_num2, std::forward(f)); + + case TypeIndex::Enum8: return callOnBasicType(type_num2, std::forward(f)); + case TypeIndex::Enum16: return callOnBasicType(type_num2, std::forward(f)); + default: break; } From 4c3af43ed34c3ec8c730fbe8ebefb00e6d58471d Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 19 Dec 2018 17:36:55 +0300 Subject: [PATCH 085/181] Add test. --- .../queries/0_stateless/00806_alter_update.reference | 3 +++ dbms/tests/queries/0_stateless/00806_alter_update.sql | 11 +++++++++++ 2 files changed, 14 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00806_alter_update.reference create mode 100644 dbms/tests/queries/0_stateless/00806_alter_update.sql diff --git a/dbms/tests/queries/0_stateless/00806_alter_update.reference b/dbms/tests/queries/0_stateless/00806_alter_update.reference new file mode 100644 index 00000000000..fbc804c35ca --- /dev/null +++ b/dbms/tests/queries/0_stateless/00806_alter_update.reference @@ -0,0 +1,3 @@ +0 +foo +foo diff --git a/dbms/tests/queries/0_stateless/00806_alter_update.sql b/dbms/tests/queries/0_stateless/00806_alter_update.sql new file mode 100644 index 00000000000..65e98188c3f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00806_alter_update.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS test.alter_update; + +CREATE TABLE test.alter_update (d Date, e Enum8('foo'=1, 'bar'=2)) Engine = MergeTree(d, (d), 8192); +INSERT INTO test.alter_update (d, e) VALUES ('2018-01-01', 'foo'); +INSERT INTO test.alter_update (d, e) VALUES ('2018-01-02', 'bar'); + +ALTER TABLE test.alter_update UPDATE e = CAST('foo', 'Enum8(\'foo\' = 1, \'bar\' = 2)') WHERE d='2018-01-02'; + +SELECT sleep(1); -- TODO: there should be setting for sync ALTER UPDATE someday. + +SELECT e FROM test.alter_update ORDER BY d; From 24a0b3a53e033815a5f31cdd9060bb8484364dd5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 19 Dec 2018 17:46:18 +0300 Subject: [PATCH 086/181] Additional function case insensitivity from #3704 --- dbms/src/Functions/FunctionsStringSearch.cpp | 2 +- dbms/src/Functions/abs.cpp | 2 +- dbms/src/Functions/rand.cpp | 2 +- dbms/src/Functions/reverse.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 337edbbc168..af7ea515f4e 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -1080,7 +1080,7 @@ void registerFunctionsStringSearch(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/Functions/abs.cpp b/dbms/src/Functions/abs.cpp index e94be787cc4..872c8404176 100644 --- a/dbms/src/Functions/abs.cpp +++ b/dbms/src/Functions/abs.cpp @@ -48,7 +48,7 @@ template <> struct FunctionUnaryArithmeticMonotonicity void registerFunctionAbs(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/Functions/rand.cpp b/dbms/src/Functions/rand.cpp index cd4ced96b7e..333396c1ecd 100644 --- a/dbms/src/Functions/rand.cpp +++ b/dbms/src/Functions/rand.cpp @@ -9,7 +9,7 @@ using FunctionRand = FunctionRandom; void registerFunctionRand(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); } } diff --git a/dbms/src/Functions/reverse.cpp b/dbms/src/Functions/reverse.cpp index 065e1d28073..b7447a7882b 100644 --- a/dbms/src/Functions/reverse.cpp +++ b/dbms/src/Functions/reverse.cpp @@ -147,7 +147,7 @@ private: void registerFunctionReverse(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); } } From 98a798aa52506a6e60d17da30eb7ff4f6550605f Mon Sep 17 00:00:00 2001 From: Michal Lisowski Date: Wed, 19 Dec 2018 16:24:27 +0100 Subject: [PATCH 087/181] Mention about nagios check in 3rd party integrations (#3878) --- docs/en/interfaces/third-party/integrations.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 8d4714a178a..0da0fcf14a8 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -28,6 +28,8 @@ - [Prometheus](https://prometheus.io/) - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) + - [Nagios](https://www.nagios.org/) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) - Logging - [fluentd](https://www.fluentd.org) - [loghouse](https://github.com/flant/loghouse) (for [Kubernetes](https://kubernetes.io)) From 92499c725dbf2139cfda6f4ebba27b270f14f53d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Dec 2018 18:27:57 +0300 Subject: [PATCH 088/181] Fixed test #3811 --- .../queries/0_stateless/00800_function_java_hash.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00800_function_java_hash.sql b/dbms/tests/queries/0_stateless/00800_function_java_hash.sql index 220c1f0ff60..c69cd412f57 100644 --- a/dbms/tests/queries/0_stateless/00800_function_java_hash.sql +++ b/dbms/tests/queries/0_stateless/00800_function_java_hash.sql @@ -1,4 +1,4 @@ -select JavaHash('abc'); -select JavaHash('874293087'); -select HiveHash('abc'); -select HiveHash('874293087'); +select javaHash('abc'); +select javaHash('874293087'); +select hiveHash('abc'); +select hiveHash('874293087'); From 214b500b6e24b04f02d59fd60353a4ff5fe4b1bf Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 18:30:10 +0300 Subject: [PATCH 089/181] Fix build without submodules --- cmake/find_libgsasl.cmake | 3 ++- cmake/find_libxml2.cmake | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/cmake/find_libgsasl.cmake b/cmake/find_libgsasl.cmake index 0c88bd2c24a..f22ead4c765 100644 --- a/cmake/find_libgsasl.cmake +++ b/cmake/find_libgsasl.cmake @@ -5,6 +5,7 @@ endif () if (USE_INTERNAL_LIBGSASL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_LIBGSASL_LIBRARY 0) + set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) @@ -13,7 +14,7 @@ if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) endif () if (LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) -elseif (NOT APPLE AND NOT ARCH_32) +elseif (NOT MISSING_INTERNAL_LIBGSASL_LIBRARY AND NOT APPLE AND NOT ARCH_32) set (LIBGSASL_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include) set (USE_INTERNAL_LIBGSASL_LIBRARY 1) set (LIBGSASL_LIBRARY libgsasl) diff --git a/cmake/find_libxml2.cmake b/cmake/find_libxml2.cmake index cfababfbf63..db34801b238 100644 --- a/cmake/find_libxml2.cmake +++ b/cmake/find_libxml2.cmake @@ -3,6 +3,7 @@ option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library if (USE_INTERNAL_LIBXML2_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_LIBXML2_LIBRARY 0) + set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBXML2_LIBRARY) @@ -11,7 +12,7 @@ if (NOT USE_INTERNAL_LIBXML2_LIBRARY) endif () if (LIBXML2_LIBRARY AND LIBXML2_INCLUDE_DIR) -else () +elseif (NOT MISSING_INTERNAL_LIBXML2_LIBRARY) set (LIBXML2_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libxml2/include ${ClickHouse_SOURCE_DIR}/contrib/libxml2-cmake/linux_x86_64/include) set (USE_INTERNAL_LIBXML2_LIBRARY 1) set (LIBXML2_LIBRARY libxml2) From bb3487d77891cc4d3f9da1fde25a26644dbb9407 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 18:36:31 +0300 Subject: [PATCH 090/181] Fix build without submodules --- cmake/find_hdfs3.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find_hdfs3.cmake b/cmake/find_hdfs3.cmake index a30409b50d5..81fe7177304 100644 --- a/cmake/find_hdfs3.cmake +++ b/cmake/find_hdfs3.cmake @@ -16,7 +16,7 @@ endif () if (HDFS3_LIBRARY AND HDFS3_INCLUDE_DIR) set(USE_HDFS 1) -elseif (LIBGSASL_LIBRARY) +elseif (LIBGSASL_LIBRARY AND LIBXML2_LIBRARY) set(HDFS3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include") set(HDFS3_LIBRARY hdfs3) set(USE_HDFS 1) From b56dea9f84dc29c3c0261c25980648d161eaf500 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 18:37:55 +0300 Subject: [PATCH 091/181] Fix build without submodules --- cmake/find_hdfs3.cmake | 2 +- cmake/find_libgsasl.cmake | 3 ++- cmake/find_libxml2.cmake | 3 ++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/cmake/find_hdfs3.cmake b/cmake/find_hdfs3.cmake index a30409b50d5..81fe7177304 100644 --- a/cmake/find_hdfs3.cmake +++ b/cmake/find_hdfs3.cmake @@ -16,7 +16,7 @@ endif () if (HDFS3_LIBRARY AND HDFS3_INCLUDE_DIR) set(USE_HDFS 1) -elseif (LIBGSASL_LIBRARY) +elseif (LIBGSASL_LIBRARY AND LIBXML2_LIBRARY) set(HDFS3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include") set(HDFS3_LIBRARY hdfs3) set(USE_HDFS 1) diff --git a/cmake/find_libgsasl.cmake b/cmake/find_libgsasl.cmake index 0c88bd2c24a..f22ead4c765 100644 --- a/cmake/find_libgsasl.cmake +++ b/cmake/find_libgsasl.cmake @@ -5,6 +5,7 @@ endif () if (USE_INTERNAL_LIBGSASL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_LIBGSASL_LIBRARY 0) + set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) @@ -13,7 +14,7 @@ if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) endif () if (LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR) -elseif (NOT APPLE AND NOT ARCH_32) +elseif (NOT MISSING_INTERNAL_LIBGSASL_LIBRARY AND NOT APPLE AND NOT ARCH_32) set (LIBGSASL_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include) set (USE_INTERNAL_LIBGSASL_LIBRARY 1) set (LIBGSASL_LIBRARY libgsasl) diff --git a/cmake/find_libxml2.cmake b/cmake/find_libxml2.cmake index cfababfbf63..db34801b238 100644 --- a/cmake/find_libxml2.cmake +++ b/cmake/find_libxml2.cmake @@ -3,6 +3,7 @@ option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library if (USE_INTERNAL_LIBXML2_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") set (USE_INTERNAL_LIBXML2_LIBRARY 0) + set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBXML2_LIBRARY) @@ -11,7 +12,7 @@ if (NOT USE_INTERNAL_LIBXML2_LIBRARY) endif () if (LIBXML2_LIBRARY AND LIBXML2_INCLUDE_DIR) -else () +elseif (NOT MISSING_INTERNAL_LIBXML2_LIBRARY) set (LIBXML2_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libxml2/include ${ClickHouse_SOURCE_DIR}/contrib/libxml2-cmake/linux_x86_64/include) set (USE_INTERNAL_LIBXML2_LIBRARY 1) set (LIBXML2_LIBRARY libxml2) From d5dbfd57418d5235ccfadc7931cdf03f1343a0b0 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 19:15:29 +0300 Subject: [PATCH 092/181] Fix build without submodules --- cmake/find_libgsasl.cmake | 10 ++++++---- cmake/find_libxml2.cmake | 10 ++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/cmake/find_libgsasl.cmake b/cmake/find_libgsasl.cmake index f22ead4c765..ef1bbefe0df 100644 --- a/cmake/find_libgsasl.cmake +++ b/cmake/find_libgsasl.cmake @@ -2,10 +2,12 @@ if (NOT APPLE AND NOT ARCH_32) option (USE_INTERNAL_LIBGSASL_LIBRARY "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED}) endif () -if (USE_INTERNAL_LIBGSASL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") - message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_LIBGSASL_LIBRARY 0) - set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") + if (USE_INTERNAL_LIBGSASL_LIBRARY) + message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_LIBGSASL_LIBRARY 0) + endif () + set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) diff --git a/cmake/find_libxml2.cmake b/cmake/find_libxml2.cmake index db34801b238..b421d113361 100644 --- a/cmake/find_libxml2.cmake +++ b/cmake/find_libxml2.cmake @@ -1,9 +1,11 @@ option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library instead of bundled" ${NOT_UNBUNDLED}) -if (USE_INTERNAL_LIBXML2_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") - message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_LIBXML2_LIBRARY 0) - set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") + if (USE_INTERNAL_LIBXML2_LIBRARY) + message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_LIBXML2_LIBRARY 0) + endif () + set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBXML2_LIBRARY) From 133b7240a37209401942b1e2e970318ac2677ce5 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 19:15:29 +0300 Subject: [PATCH 093/181] Fix build without submodules --- cmake/find_libgsasl.cmake | 10 ++++++---- cmake/find_libxml2.cmake | 10 ++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/cmake/find_libgsasl.cmake b/cmake/find_libgsasl.cmake index f22ead4c765..ef1bbefe0df 100644 --- a/cmake/find_libgsasl.cmake +++ b/cmake/find_libgsasl.cmake @@ -2,10 +2,12 @@ if (NOT APPLE AND NOT ARCH_32) option (USE_INTERNAL_LIBGSASL_LIBRARY "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED}) endif () -if (USE_INTERNAL_LIBGSASL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") - message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_LIBGSASL_LIBRARY 0) - set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") + if (USE_INTERNAL_LIBGSASL_LIBRARY) + message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_LIBGSASL_LIBRARY 0) + endif () + set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBGSASL_LIBRARY) diff --git a/cmake/find_libxml2.cmake b/cmake/find_libxml2.cmake index db34801b238..b421d113361 100644 --- a/cmake/find_libxml2.cmake +++ b/cmake/find_libxml2.cmake @@ -1,9 +1,11 @@ option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library instead of bundled" ${NOT_UNBUNDLED}) -if (USE_INTERNAL_LIBXML2_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") - message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_LIBXML2_LIBRARY 0) - set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") + if (USE_INTERNAL_LIBXML2_LIBRARY) + message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_LIBXML2_LIBRARY 0) + endif () + set (MISSING_INTERNAL_LIBXML2_LIBRARY 1) endif () if (NOT USE_INTERNAL_LIBXML2_LIBRARY) From 6959dc3b335790904ba28f19694449ebb0fcd7d6 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 19:25:38 +0300 Subject: [PATCH 094/181] Fix build without submodules --- cmake/find_base64.cmake | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/cmake/find_base64.cmake b/cmake/find_base64.cmake index ad71d11c1b4..9b6e28a8ccf 100644 --- a/cmake/find_base64.cmake +++ b/cmake/find_base64.cmake @@ -1,4 +1,11 @@ -option (ENABLE_BASE64 "Enable base64" ON) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/base64/lib/lib.c") + set (MISSING_INTERNAL_BASE64_LIBRARY 1) + message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") +endif () + +if (NOT MISSING_INTERNAL_BASE64_LIBRARY) + option (ENABLE_BASE64 "Enable base64" ON) +endif () if (ENABLE_BASE64) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") @@ -9,4 +16,3 @@ if (ENABLE_BASE64) set (USE_BASE64 1) endif() endif () - From bf625b25c3f0cde70bdbb88451620d0a720f4a68 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 19:25:38 +0300 Subject: [PATCH 095/181] Fix build without submodules --- cmake/find_base64.cmake | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/cmake/find_base64.cmake b/cmake/find_base64.cmake index ad71d11c1b4..9b6e28a8ccf 100644 --- a/cmake/find_base64.cmake +++ b/cmake/find_base64.cmake @@ -1,4 +1,11 @@ -option (ENABLE_BASE64 "Enable base64" ON) +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/base64/lib/lib.c") + set (MISSING_INTERNAL_BASE64_LIBRARY 1) + message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") +endif () + +if (NOT MISSING_INTERNAL_BASE64_LIBRARY) + option (ENABLE_BASE64 "Enable base64" ON) +endif () if (ENABLE_BASE64) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") @@ -9,4 +16,3 @@ if (ENABLE_BASE64) set (USE_BASE64 1) endif() endif () - From 78321a04a1aea7ff9acc8b8cf9a02f2884886ed6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 19 Dec 2018 19:47:30 +0300 Subject: [PATCH 096/181] Added low_cardinality_allow_in_native_format setting. --- dbms/programs/server/TCPHandler.cpp | 9 ++++++--- dbms/src/DataStreams/NativeBlockInputStream.cpp | 7 ++++--- dbms/src/DataStreams/NativeBlockInputStream.h | 4 +++- dbms/src/DataStreams/NativeBlockOutputStream.cpp | 6 +++--- dbms/src/DataStreams/NativeBlockOutputStream.h | 4 +++- dbms/src/Interpreters/Settings.h | 2 +- 6 files changed, 20 insertions(+), 12 deletions(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index cfb0cd3cd58..6f74b30ea02 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -782,7 +782,8 @@ void TCPHandler::initBlockInput() state.block_in = std::make_shared( *state.maybe_compressed_in, header, - client_revision); + client_revision, + !connection_context.getSettingsRef().low_cardinality_allow_in_native_format); } } @@ -803,7 +804,8 @@ void TCPHandler::initBlockOutput(const Block & block) state.block_out = std::make_shared( *state.maybe_compressed_out, client_revision, - block.cloneEmpty()); + block.cloneEmpty(), + !connection_context.getSettingsRef().low_cardinality_allow_in_native_format); } } @@ -815,7 +817,8 @@ void TCPHandler::initLogsBlockOutput(const Block & block) state.logs_block_out = std::make_shared( *out, client_revision, - block.cloneEmpty()); + block.cloneEmpty(), + !connection_context.getSettingsRef().low_cardinality_allow_in_native_format); } } diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 7cd4a571a60..7eeba3b9e50 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -29,8 +29,8 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server { } -NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_) - : istr(istr_), header(header_), server_revision(server_revision_) +NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_, bool convert_types_to_low_cardinality_) + : istr(istr_), header(header_), server_revision(server_revision_), convert_types_to_low_cardinality(convert_types_to_low_cardinality_) { } @@ -154,7 +154,8 @@ Block NativeBlockInputStream::readImpl() column.column = std::move(read_column); /// Support insert from old clients without low cardinality type. - if (header && server_revision && server_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) + bool revision_without_low_cardinality = server_revision && server_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE; + if (header && (convert_types_to_low_cardinality || revision_without_low_cardinality)) { column.column = recursiveLowCardinalityConversion(column.column, column.type, header.getByPosition(i).type); column.type = header.getByPosition(i).type; diff --git a/dbms/src/DataStreams/NativeBlockInputStream.h b/dbms/src/DataStreams/NativeBlockInputStream.h index f16b8c4a595..51526d58422 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.h +++ b/dbms/src/DataStreams/NativeBlockInputStream.h @@ -65,7 +65,7 @@ public: /// For cases when data structure (header) is known in advance. /// NOTE We may use header for data validation and/or type conversions. It is not implemented. - NativeBlockInputStream(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_); + NativeBlockInputStream(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_, bool convert_types_to_low_cardinality_ = false); /// For cases when we have an index. It allows to skip columns. Only columns specified in the index will be read. NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_, @@ -90,6 +90,8 @@ private: IndexForNativeFormat::Blocks::const_iterator index_block_it; IndexForNativeFormat::Blocks::const_iterator index_block_end; IndexOfBlockForNativeFormat::Columns::const_iterator index_column_it; + + bool convert_types_to_low_cardinality = false; /// If an index is specified, then `istr` must be CompressedReadBufferFromFile. Unused otherwise. CompressedReadBufferFromFile * istr_concrete = nullptr; diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index c87d82b2506..1869badfe14 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -21,10 +21,10 @@ namespace ErrorCodes NativeBlockOutputStream::NativeBlockOutputStream( - WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, + WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_, WriteBuffer * index_ostr_, size_t initial_size_of_file_) : ostr(ostr_), client_revision(client_revision_), header(header_), - index_ostr(index_ostr_), initial_size_of_file(initial_size_of_file_) + index_ostr(index_ostr_), initial_size_of_file(initial_size_of_file_), remove_low_cardinality(remove_low_cardinality_) { if (index_ostr) { @@ -104,7 +104,7 @@ void NativeBlockOutputStream::write(const Block & block) ColumnWithTypeAndName column = block.safeGetByPosition(i); /// Send data to old clients without low cardinality type. - if (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) + if (remove_low_cardinality || (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE)) { column.column = recursiveRemoveLowCardinality(column.column); column.type = recursiveRemoveLowCardinality(column.type); diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.h b/dbms/src/DataStreams/NativeBlockOutputStream.h index 7e3f14e06ea..b4dfc85073d 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.h +++ b/dbms/src/DataStreams/NativeBlockOutputStream.h @@ -23,7 +23,7 @@ public: /** If non-zero client_revision is specified, additional block information can be written. */ NativeBlockOutputStream( - WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, + WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_ = false, WriteBuffer * index_ostr_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const override { return header; } @@ -42,6 +42,8 @@ private: size_t initial_size_of_file; /// The initial size of the data file, if `append` done. Used for the index. /// If you need to write index, then `ostr` must be a CompressedWriteBuffer. CompressedWriteBuffer * ostr_concrete = nullptr; + + bool remove_low_cardinality; }; } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index d5775dd3945..e6ac4c356eb 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -294,7 +294,7 @@ struct Settings M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \ M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.") \ M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.") \ - + M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise convert LowCardinality columns to ordinary.") \ #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; From 0f8f5836fb685bd295682d8cb9a52c94714849ef Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 19 Dec 2018 19:55:01 +0300 Subject: [PATCH 097/181] Fix build. --- dbms/src/Storages/StorageStripeLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 018001db8dc..b86731f6f3f 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -139,7 +139,7 @@ public: data_out(data_out_compressed, CompressionSettings(CompressionMethod::LZ4), storage.max_compress_block_size), index_out_compressed(storage.full_path() + "index.mrk", INDEX_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), index_out(index_out_compressed), - block_out(data_out, 0, storage.getSampleBlock(), &index_out, Poco::File(storage.full_path() + "data.bin").getSize()) + block_out(data_out, 0, storage.getSampleBlock(), false, &index_out, Poco::File(storage.full_path() + "data.bin").getSize()) { } From 4eb7a820d6710c83df4738c7b4f3cb6946f9c990 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 19 Dec 2018 20:54:46 +0300 Subject: [PATCH 098/181] Fix style --- dbms/src/DataStreams/NativeBlockInputStream.h | 2 +- dbms/src/DataStreams/NativeBlockOutputStream.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/NativeBlockInputStream.h b/dbms/src/DataStreams/NativeBlockInputStream.h index 51526d58422..c9b565cd9d7 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.h +++ b/dbms/src/DataStreams/NativeBlockInputStream.h @@ -90,7 +90,7 @@ private: IndexForNativeFormat::Blocks::const_iterator index_block_it; IndexForNativeFormat::Blocks::const_iterator index_block_end; IndexOfBlockForNativeFormat::Columns::const_iterator index_column_it; - + bool convert_types_to_low_cardinality = false; /// If an index is specified, then `istr` must be CompressedReadBufferFromFile. Unused otherwise. diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.h b/dbms/src/DataStreams/NativeBlockOutputStream.h index b4dfc85073d..9834b90ef2a 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.h +++ b/dbms/src/DataStreams/NativeBlockOutputStream.h @@ -42,7 +42,7 @@ private: size_t initial_size_of_file; /// The initial size of the data file, if `append` done. Used for the index. /// If you need to write index, then `ostr` must be a CompressedWriteBuffer. CompressedWriteBuffer * ostr_concrete = nullptr; - + bool remove_low_cardinality; }; From fdd22ad187340925858407d32fb3aa8c68012070 Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 20:56:58 +0300 Subject: [PATCH 099/181] Fix includes --- dbms/programs/server/TCPHandler.h | 1 + dbms/src/Interpreters/RequiredSourceColumnsVisitor.h | 1 + dbms/src/Parsers/ASTEnumElement.h | 4 ++-- utils/check-style/check-include | 3 +++ 4 files changed, 7 insertions(+), 2 deletions(-) diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index 43c544cc401..98b76268047 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -25,6 +25,7 @@ namespace Poco { class Logger; } namespace DB { +class ColumnsDescription; /// State of query processing. struct QueryState diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h index ebf948ffeb8..859326acbe8 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -8,6 +8,7 @@ #include #include #include +#include "InDepthNodeVisitor.h" namespace DB { diff --git a/dbms/src/Parsers/ASTEnumElement.h b/dbms/src/Parsers/ASTEnumElement.h index 8e0a2ec1648..c603f5086de 100644 --- a/dbms/src/Parsers/ASTEnumElement.h +++ b/dbms/src/Parsers/ASTEnumElement.h @@ -1,6 +1,6 @@ #pragma once -#include +#include "IAST.h" #include #include @@ -18,7 +18,7 @@ public: ASTEnumElement(const String & name, const Field & value) : name{name}, value {value} {} - String getID() const override { return "EnumElement"; } + String getID(char) const override { return "EnumElement"; } ASTPtr clone() const override { diff --git a/utils/check-style/check-include b/utils/check-style/check-include index cdc5a82d7ed..eede9940dcd 100755 --- a/utils/check-style/check-include +++ b/utils/check-style/check-include @@ -34,6 +34,9 @@ inc="-I. \ -I./contrib/poco/Foundation/include \ -I./contrib/boost/libs/*/include \ -I./contrib/boost \ +-I/usr/include/llvm-7 \ +-I/usr/include/llvm-6.0 \ +-I/usr/include/llvm-5.0 \ -I./contrib/llvm/llvm/include \ -I${BUILD_DIR}/contrib/llvm/llvm/include \ -I./contrib/libbtrie/include \ From 6869771ae4438d547ada9250ae5ef42a834af4bc Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 19 Dec 2018 20:57:51 +0300 Subject: [PATCH 100/181] Fix includes --- dbms/programs/server/TCPHandler.h | 1 + dbms/src/Interpreters/RequiredSourceColumnsVisitor.h | 1 + dbms/src/Parsers/ASTEnumElement.h | 4 ++-- utils/check-style/check-include | 3 +++ 4 files changed, 7 insertions(+), 2 deletions(-) diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index 43c544cc401..98b76268047 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -25,6 +25,7 @@ namespace Poco { class Logger; } namespace DB { +class ColumnsDescription; /// State of query processing. struct QueryState diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h index ebf948ffeb8..859326acbe8 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -8,6 +8,7 @@ #include #include #include +#include "InDepthNodeVisitor.h" namespace DB { diff --git a/dbms/src/Parsers/ASTEnumElement.h b/dbms/src/Parsers/ASTEnumElement.h index 8e0a2ec1648..c603f5086de 100644 --- a/dbms/src/Parsers/ASTEnumElement.h +++ b/dbms/src/Parsers/ASTEnumElement.h @@ -1,6 +1,6 @@ #pragma once -#include +#include "IAST.h" #include #include @@ -18,7 +18,7 @@ public: ASTEnumElement(const String & name, const Field & value) : name{name}, value {value} {} - String getID() const override { return "EnumElement"; } + String getID(char) const override { return "EnumElement"; } ASTPtr clone() const override { diff --git a/utils/check-style/check-include b/utils/check-style/check-include index cdc5a82d7ed..eede9940dcd 100755 --- a/utils/check-style/check-include +++ b/utils/check-style/check-include @@ -34,6 +34,9 @@ inc="-I. \ -I./contrib/poco/Foundation/include \ -I./contrib/boost/libs/*/include \ -I./contrib/boost \ +-I/usr/include/llvm-7 \ +-I/usr/include/llvm-6.0 \ +-I/usr/include/llvm-5.0 \ -I./contrib/llvm/llvm/include \ -I${BUILD_DIR}/contrib/llvm/llvm/include \ -I./contrib/libbtrie/include \ From acb8cf1849b677753237e7faac9c5f41b78665a8 Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Wed, 19 Dec 2018 23:29:52 +0300 Subject: [PATCH 101/181] Fix test. --- .../configs/config.xml | 3 +++ .../configs/users.xml | 20 +++++++++++++++ .../test.py | 25 +++++++++++-------- 3 files changed, 38 insertions(+), 10 deletions(-) diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml index 48aa82349d3..032b8874af2 100644 --- a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml @@ -7,5 +7,8 @@ /var/log/clickhouse-server/stdout.log + /var/lib/clickhouse/ + + 5368709120 users.xml diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml index 9aba4ac0914..6061af8e33d 100644 --- a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/users.xml @@ -1,3 +1,23 @@ + + + + + + + + + + ::/0 + + default + default + + + + + + + diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py b/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py index 56c40530c36..c96560230eb 100644 --- a/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -1,3 +1,4 @@ +import docker import os import pwd import pytest @@ -6,32 +7,36 @@ import re from helpers.cluster import ClickHouseCluster -def expect_failure_with_message(config, expected_message): +def expect_failure_with_message(expected_message): cluster = ClickHouseCluster(__file__) - node = cluster.add_instance('node', main_configs=[config], with_zookeeper=False) + node = cluster.add_instance('node', with_zookeeper=False) with pytest.raises(Exception): cluster.start() + current_user_id = os.getuid() + other_user_id = pwd.getpwnam('nobody').pw_uid + + docker_api = docker.from_env().api + container = node.get_docker_handle() + container.start() + container.exec_run('chown {0} /var/lib/clickhouse'.format(other_user_id), privileged=True) + container.exec_run('clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log', privileged=True) + cluster.shutdown() # cleanup - with open(os.path.join(node.path, 'logs/stderr.log')) as log: + with open(os.path.join(node.path, 'logs/clickhouse-server.err.log')) as log: last_message = log.readlines()[-1].strip() if re.search(expected_message, last_message) is None: pytest.fail('Expected the server to fail with a message "{}", but the last message is "{}"'.format(expected_message, last_message)) -def test_no_such_directory(): - expect_failure_with_message('configs/no_such_directory.xml', 'Failed to stat.*no_such_directory') - - def test_different_user(): current_user_id = os.getuid() - if current_user_id != 0: + if current_user_id == 0: current_user_name = pwd.getpwuid(current_user_id).pw_name expect_failure_with_message( - 'configs/owner_mismatch.xml', - 'Effective user of the process \(({}|{})\) does not match the owner of the data \((0|root)\)'.format(current_user_id, current_user_name), + 'Effective user of the process \(.*\) does not match the owner of the data \(.*\)', ) From 81eeaec7fe76b2e054bb26de5cb8f7acb536a917 Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Thu, 20 Dec 2018 00:12:06 +0300 Subject: [PATCH 102/181] Cleanup test. --- .../configs/no_such_directory.xml | 3 --- .../configs/owner_mismatch.xml | 3 --- 2 files changed, 6 deletions(-) delete mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml delete mode 100644 dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml deleted file mode 100644 index 80ddf7c4722..00000000000 --- a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/no_such_directory.xml +++ /dev/null @@ -1,3 +0,0 @@ - - /no_such_directory/data/ - diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml deleted file mode 100644 index 46d2dcc49ee..00000000000 --- a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/owner_mismatch.xml +++ /dev/null @@ -1,3 +0,0 @@ - - /root/data/ - From fb5dbb7959697970342f6dddce2d64e82980aac3 Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Thu, 20 Dec 2018 01:47:42 +0300 Subject: [PATCH 103/181] Do not require the server to fail to start in the test. --- .../configs/config.xml | 3 +++ .../test_match_process_uid_against_data_owner/test.py | 10 +++++----- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml index 032b8874af2..1bd6dc45549 100644 --- a/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/configs/config.xml @@ -7,6 +7,9 @@ /var/log/clickhouse-server/stdout.log + 9000 + 127.0.0.1 + /var/lib/clickhouse/ 5368709120 diff --git a/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py b/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py index 080066f5b0a..77cbe25ff80 100644 --- a/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py +++ b/dbms/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -13,16 +13,16 @@ def test_different_user(): if current_user_id != 0: return + other_user_id = pwd.getpwnam('nobody').pw_uid + cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node') - with pytest.raises(Exception): - cluster.start() - - other_user_id = pwd.getpwnam('nobody').pw_uid + cluster.start() docker_api = docker.from_env().api container = node.get_docker_handle() + container.stop() container.start() container.exec_run('chown {} /var/lib/clickhouse'.format(other_user_id), privileged=True) container.exec_run(CLICKHOUSE_START_COMMAND) @@ -30,8 +30,8 @@ def test_different_user(): cluster.shutdown() # cleanup with open(os.path.join(node.path, 'logs/clickhouse-server.err.log')) as log: + expected_message = "Effective user of the process \(.*\) does not match the owner of the data \(.*\)\. Run under 'sudo -u .*'\." last_message = log.readlines()[-1].strip() - expected_message = 'Effective user of the process \(.*\) does not match the owner of the data \(.*\)' if re.search(expected_message, last_message) is None: pytest.fail('Expected the server to fail with a message "{}", but the last message is "{}"'.format(expected_message, last_message)) From dc61a73a6eb5f7f0ef35459f8bdbb0686f419726 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 19 Dec 2018 14:57:37 +0800 Subject: [PATCH 104/181] Fixed convert function monotonic failure when string data type --- dbms/src/Functions/FunctionsConversion.h | 4 ++++ .../00653_monotonic_integer_cast.reference | 1 + .../0_stateless/00653_monotonic_integer_cast.sql | 15 ++++++++++----- 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 1428fec4f48..bd7cb6396ec 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1138,6 +1138,10 @@ struct ToIntMonotonicity static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) { + /// If type is string, the function is no monotonic + if (WhichDataType(type).isString()) + return {}; + size_t size_of_type = type.getSizeOfValueInMemory(); /// If type is expanding diff --git a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference index d00491fd7e5..6ed281c757a 100644 --- a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference +++ b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference @@ -1 +1,2 @@ 1 +1 diff --git a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql index 99025e59b89..f681b024082 100644 --- a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql +++ b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql @@ -1,5 +1,10 @@ -drop table if exists test.table; -create table test.table (val Int32) engine = MergeTree order by val; -insert into test.table values (-2), (0), (2); -select count() from test.table where toUInt64(val) == 0; - +DROP TABLE IF EXISTS test.number_test_table; +DROP TABLE IF EXISTS test.string_test_table; +CREATE TABLE test.number_test_table (val Int32) ENGINE = MergeTree ORDER BY val; +CREATE TABLE test.string_test_table (val String) ENGINE = MergeTree ORDER BY val; +INSERT INTO test.number_test_table VALUES (-2), (0), (2); +INSERT INTO test.string_test_table VALUES ('0'), ('2'); +SELECT count() FROM test.number_test_table WHERE toUInt64(val) == 0; +SELECT count() FROM test.string_test_table WHERE toUInt64(val) == 0; +DROP TABLE IF EXISTS test.number_test_table; +DROP TABLE IF EXISTS test.string_test_table; From f63aa0eb3bc7acaa97885b3594bd3424d28de743 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 20 Dec 2018 11:22:24 +0300 Subject: [PATCH 105/181] Support MySQL-style syntax for DATE_ADD and similar functions --- dbms/src/Parsers/ExpressionElementParsers.cpp | 53 +++++++++++++------ .../00765_sql_compatibility_aliases.reference | 1 + .../00765_sql_compatibility_aliases.sql | 1 + 3 files changed, 38 insertions(+), 17 deletions(-) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 35e66da5141..d01dfd1b7a8 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -717,8 +717,8 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { const char * function_name = nullptr; - ASTPtr left_node; - ASTPtr right_node; + ASTPtr timestamp_node; + ASTPtr offset_node; if (ParserKeyword("DATEADD").ignore(pos, expected) || ParserKeyword("DATE_ADD").ignore(pos, expected) || ParserKeyword("TIMESTAMPADD").ignore(pos, expected) || ParserKeyword("TIMESTAMP_ADD").ignore(pos, expected)) @@ -734,31 +734,50 @@ bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp ++pos; ParserInterval interval_parser; - if (!interval_parser.ignore(pos, expected)) - return false; + if (interval_parser.ignore(pos, expected)) + { + /// function(unit, offset, timestamp) + if (pos->type != TokenType::Comma) + return false; + ++pos; - const char * interval_function_name = interval_parser.getToIntervalKindFunctionName(); + if (!ParserExpression().parse(pos, offset_node, expected)) + return false; - if (pos->type != TokenType::Comma) - return false; - ++pos; + if (pos->type != TokenType::Comma) + return false; + ++pos; - if (!ParserExpression().parse(pos, left_node, expected)) - return false; + if (!ParserExpression().parse(pos, timestamp_node, expected)) + return false; + } + else + { + /// function(timestamp, INTERVAL offset unit) + if (!ParserExpression().parse(pos, timestamp_node, expected)) + return false; - if (pos->type != TokenType::Comma) - return false; - ++pos; + if (pos->type != TokenType::Comma) + return false; + ++pos; - if (!ParserExpression().parse(pos, right_node, expected)) - return false; + if (!ParserKeyword("INTERVAL").ignore(pos, expected)) + return false; + if (!ParserExpression().parse(pos, offset_node, expected)) + return false; + + interval_parser.ignore(pos, expected); + + } if (pos->type != TokenType::ClosingRoundBracket) return false; ++pos; + const char * interval_function_name = interval_parser.getToIntervalKindFunctionName(); + auto interval_expr_list_args = std::make_shared(); - interval_expr_list_args->children = {left_node}; + interval_expr_list_args->children = {offset_node}; auto interval_func_node = std::make_shared(); interval_func_node->name = interval_function_name; @@ -766,7 +785,7 @@ bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp interval_func_node->children.push_back(interval_func_node->arguments); auto expr_list_args = std::make_shared(); - expr_list_args->children = {right_node, interval_func_node}; + expr_list_args->children = {timestamp_node, interval_func_node}; auto func_node = std::make_shared(); func_node->name = function_name; diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference index 43e7758e0f1..6a2a0523476 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.reference @@ -25,3 +25,4 @@ foo* -3 2021-01-01 2018-07-18 01:02:03 +2018-04-01 diff --git a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql index 6373d579f39..a7f1f3ad98a 100644 --- a/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql +++ b/dbms/tests/queries/0_stateless/00765_sql_compatibility_aliases.sql @@ -27,3 +27,4 @@ select DATE_DIFF(MONTH, toDate('2018-12-18'), toDate('2018-01-01')); select DATE_DIFF(QQ, toDate('2018-12-18'), toDate('2018-01-01')); select DATE_ADD(YEAR, 3, toDate('2018-01-01')); select timestamp_sub(SQL_TSI_MONTH, 5, toDateTime('2018-12-18 01:02:03')); +select timestamp_ADD(toDate('2018-01-01'), INTERVAL 3 MONTH); From bd179822248094a38576d18a2b9cf6e7988281b0 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 20 Dec 2018 12:54:28 +0300 Subject: [PATCH 106/181] Fix clang build --- dbms/programs/server/TCPHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index 98b76268047..19641e88d25 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -25,7 +25,7 @@ namespace Poco { class Logger; } namespace DB { -class ColumnsDescription; +struct ColumnsDescription; /// State of query processing. struct QueryState From b63ddfa183d16a15e2a0b7901d1118a2571ffadc Mon Sep 17 00:00:00 2001 From: ogorbacheva Date: Thu, 20 Dec 2018 15:04:28 +0300 Subject: [PATCH 107/181] changed the zookeper settings example (#3883) --- docs/en/operations/server_settings/settings.md | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 4275b5514c0..cc65063c70b 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -681,8 +681,20 @@ For more information, see the section "[Replication](../../operations/table_engi **Example** ```xml - + + + example1 + 2181 + + + example2 + 2181 + + + example3 + 2181 + + ``` - [Original article](https://clickhouse.yandex/docs/en/operations/server_settings/settings/) From 7c5cff2dd4b779e912cf36744ae5f753a49b3489 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 16:23:49 +0300 Subject: [PATCH 108/181] Renamed test #3350 --- ...ase64_functions.reference => 00732_base64_functions.reference} | 0 .../{000732_base64_functions.sql => 00732_base64_functions.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{000732_base64_functions.reference => 00732_base64_functions.reference} (100%) rename dbms/tests/queries/0_stateless/{000732_base64_functions.sql => 00732_base64_functions.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/000732_base64_functions.reference b/dbms/tests/queries/0_stateless/00732_base64_functions.reference similarity index 100% rename from dbms/tests/queries/0_stateless/000732_base64_functions.reference rename to dbms/tests/queries/0_stateless/00732_base64_functions.reference diff --git a/dbms/tests/queries/0_stateless/000732_base64_functions.sql b/dbms/tests/queries/0_stateless/00732_base64_functions.sql similarity index 100% rename from dbms/tests/queries/0_stateless/000732_base64_functions.sql rename to dbms/tests/queries/0_stateless/00732_base64_functions.sql From 4e58b62b0cdaf98bd37fb2d0ca4681003b6100e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Dec 2018 16:28:20 +0300 Subject: [PATCH 109/181] Added LowCardinality support for specialized aggregation. --- dbms/src/Interpreters/Aggregator.cpp | 1 - dbms/src/Interpreters/SpecializedAggregator.h | 54 ++++++++++++------- 2 files changed, 36 insertions(+), 19 deletions(-) diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 1fd614affc4..03f04d791a0 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -615,7 +615,6 @@ void NO_INLINE Aggregator::executeImplCase( AggregateDataPtr overflow_row) const { /// NOTE When editing this code, also pay attention to SpecializedAggregator.h. - /// TODO for low cardinality optimization. /// For all rows. typename Method::Key prev_key; diff --git a/dbms/src/Interpreters/SpecializedAggregator.h b/dbms/src/Interpreters/SpecializedAggregator.h index 8ef66135a65..615911a7224 100644 --- a/dbms/src/Interpreters/SpecializedAggregator.h +++ b/dbms/src/Interpreters/SpecializedAggregator.h @@ -108,7 +108,10 @@ void NO_INLINE Aggregator::executeSpecialized( AggregateDataPtr overflow_row) const { typename Method::State state; - state.init(key_columns); + if constexpr (Method::low_cardinality_optimization) + state.init(key_columns, aggregation_state_cache); + else + state.init(key_columns); if (!no_more_keys) executeSpecializedCase( @@ -133,15 +136,19 @@ void NO_INLINE Aggregator::executeSpecializedCase( AggregateDataPtr overflow_row) const { /// For all rows. - typename Method::iterator it; typename Method::Key prev_key; + AggregateDataPtr value = nullptr; for (size_t i = 0; i < rows; ++i) { - bool inserted; /// Inserted a new key, or was this key already? - bool overflow = false; /// New key did not fit in the hash table because of no_more_keys. + bool inserted = false; /// Inserted a new key, or was this key already? /// Get the key to insert into the hash table. - typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool); + typename Method::Key key; + if constexpr (!Method::low_cardinality_optimization) + key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool); + + AggregateDataPtr * aggregate_data = nullptr; + typename Method::iterator it; /// Is not used if Method::low_cardinality_optimization if (!no_more_keys) /// Insert. { @@ -150,8 +157,6 @@ void NO_INLINE Aggregator::executeSpecializedCase( { if (i != 0 && key == prev_key) { - AggregateDataPtr value = Method::getAggregateData(it->second); - /// Add values into aggregate functions. AggregateFunctionsList::forEach(AggregateFunctionsUpdater( aggregate_functions, offsets_of_aggregate_states, aggregate_columns, value, i, aggregates_pool)); @@ -163,19 +168,29 @@ void NO_INLINE Aggregator::executeSpecializedCase( prev_key = key; } - method.data.emplace(key, it, inserted); + if constexpr (Method::low_cardinality_optimization) + aggregate_data = state.emplaceKeyFromRow(method.data, i, inserted, params.keys_size, keys, *aggregates_pool); + else + { + method.data.emplace(key, it, inserted); + aggregate_data = &Method::getAggregateData(it->second); + } } else { /// Add only if the key already exists. - inserted = false; - it = method.data.find(key); - if (method.data.end() == it) - overflow = true; + if constexpr (Method::low_cardinality_optimization) + aggregate_data = state.findFromRow(method.data, i); + else + { + it = method.data.find(key); + if (method.data.end() != it) + aggregate_data = &Method::getAggregateData(it->second); + } } /// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do. - if (no_more_keys && overflow && !overflow_row) + if (!aggregate_data && !overflow_row) { method.onExistingKey(key, keys, *aggregates_pool); continue; @@ -184,22 +199,25 @@ void NO_INLINE Aggregator::executeSpecializedCase( /// If a new key is inserted, initialize the states of the aggregate functions, and possibly some stuff related to the key. if (inserted) { - AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second); - aggregate_data = nullptr; + *aggregate_data = nullptr; - method.onNewKey(*it, params.keys_size, keys, *aggregates_pool); + if constexpr (!Method::low_cardinality_optimization) + method.onNewKey(*it, params.keys_size, keys, *aggregates_pool); AggregateDataPtr place = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); AggregateFunctionsList::forEach(AggregateFunctionsCreator( aggregate_functions, offsets_of_aggregate_states, place)); - aggregate_data = place; + *aggregate_data = place; + + if constexpr (Method::low_cardinality_optimization) + state.cacheAggregateData(i, place); } else method.onExistingKey(key, keys, *aggregates_pool); - AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row; + value = aggregate_data ? *aggregate_data : overflow_row; /// Add values into the aggregate functions. AggregateFunctionsList::forEach(AggregateFunctionsUpdater( From 6730f540a1892b13cca1b20baf70c3ef337e154f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 20 Dec 2018 16:40:20 +0300 Subject: [PATCH 110/181] Fix path to search base64 library --- cmake/find_base64.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find_base64.cmake b/cmake/find_base64.cmake index 9b6e28a8ccf..8e52c8463c8 100644 --- a/cmake/find_base64.cmake +++ b/cmake/find_base64.cmake @@ -1,4 +1,4 @@ -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/base64/lib/lib.c") +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64/lib/lib.c") set (MISSING_INTERNAL_BASE64_LIBRARY 1) message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") endif () From 263de501f53bb2da5d7797aeba69f6824d269ae2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 20 Dec 2018 19:42:45 +0800 Subject: [PATCH 111/181] add test & use isRepresentedByNumber --- dbms/src/Functions/FunctionsConversion.h | 9 +- .../00653_monotonic_integer_cast.reference | 1 - .../00653_monotonic_integer_cast.sql | 14 +-- ...verification_monotonic_data_load.reference | 26 ++++++ .../00653_verification_monotonic_data_load.sh | 85 +++++++++++++++++++ 5 files changed, 117 insertions(+), 18 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference create mode 100755 dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index bd7cb6396ec..e7c4edcda5b 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1138,8 +1138,7 @@ struct ToIntMonotonicity static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) { - /// If type is string, the function is no monotonic - if (WhichDataType(type).isString()) + if (!type.isValueRepresentedByNumber()) return {}; size_t size_of_type = type.getSizeOfValueInMemory(); @@ -1157,14 +1156,10 @@ struct ToIntMonotonicity } /// If type is same, too. (Enum has separate case, because it is different data type) - if (checkAndGetDataType>(&type) || + if (checkAndGetDataType>(&type) || checkAndGetDataType>(&type)) return { true, true, true }; - /// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. - if (left.isNull() || right.isNull()) - return {}; - /// If converting from float, for monotonicity, arguments must fit in range of result type. if (WhichDataType(type).isFloat()) { diff --git a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference index 6ed281c757a..d00491fd7e5 100644 --- a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference +++ b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.reference @@ -1,2 +1 @@ 1 -1 diff --git a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql index f681b024082..29a44a4aa22 100644 --- a/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql +++ b/dbms/tests/queries/0_stateless/00653_monotonic_integer_cast.sql @@ -1,10 +1,4 @@ -DROP TABLE IF EXISTS test.number_test_table; -DROP TABLE IF EXISTS test.string_test_table; -CREATE TABLE test.number_test_table (val Int32) ENGINE = MergeTree ORDER BY val; -CREATE TABLE test.string_test_table (val String) ENGINE = MergeTree ORDER BY val; -INSERT INTO test.number_test_table VALUES (-2), (0), (2); -INSERT INTO test.string_test_table VALUES ('0'), ('2'); -SELECT count() FROM test.number_test_table WHERE toUInt64(val) == 0; -SELECT count() FROM test.string_test_table WHERE toUInt64(val) == 0; -DROP TABLE IF EXISTS test.number_test_table; -DROP TABLE IF EXISTS test.string_test_table; +drop table if exists test.table; +create table test.table (val Int32) engine = MergeTree order by val; +insert into test.table values (-2), (0), (2); +select count() from test.table where toUInt64(val) == 0; diff --git a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference new file mode 100644 index 00000000000..8900af059b8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.reference @@ -0,0 +1,26 @@ +no monotonic int case: String -> UInt64 +no monotonic int case: FixedString -> UInt64 +monotonic int case: Int32 -> Int64 +monotonic int case: Int32 -> UInt64 +monotonic int case: Int32 -> Int32 +monotonic int case: Int32 -> UInt32 +monotonic int case: Int32 -> Int16 +monotonic int case: Int32 -> UInt16 +monotonic int case: UInt32 -> Int64 +monotonic int case: UInt32 -> UInt64 +monotonic int case: UInt32 -> Int32 +monotonic int case: UInt32 -> UInt32 +monotonic int case: UInt32 -> Int16 +monotonic int case: UInt32 -> UInt16 +monotonic int case: Enum16 -> Int32 +monotonic int case: Enum16 -> UInt32 +monotonic int case: Enum16 -> Int16 +monotonic int case: Enum16 -> UInt16 +monotonic int case: Enum16 -> Int8 +monotonic int case: Enum16 -> UInt8 +monotonic int case: Date -> Int32 +monotonic int case: Date -> UInt32 +monotonic int case: Date -> Int16 +monotonic int case: Date -> UInt16 +monotonic int case: Date -> Int8 +monotonic int case: Date -> UInt8 diff --git a/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh new file mode 100755 index 00000000000..325f19dc9ec --- /dev/null +++ b/dbms/tests/queries/0_stateless/00653_verification_monotonic_data_load.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash + +#-------------------------------------------- +# Description of test result: +# Test the correctness of the optimization +# by asserting read marks in the log. +# Relation of read marks and optimization: +# read marks = +# the number of monotonic marks filtered through predicates +# + no monotonic marks count +#-------------------------------------------- + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES;" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.string_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.fixed_string_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.signed_integer_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.unsigned_integer_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.enum_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.date_test_table;" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.string_test_table (val String) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.fixed_string_test_table (val FixedString(1)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.signed_integer_test_table (val Int32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.unsigned_integer_test_table (val UInt32) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.enum_test_table (val Enum16('hello' = 1, 'world' = 2, 'yandex' = 256, 'clickhouse' = 257)) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.date_test_table (val Date) ENGINE = MergeTree ORDER BY val SETTINGS index_granularity = 1;" + + +${CLICKHOUSE_CLIENT} --query="INSERT INTO test.string_test_table VALUES ('0'), ('2'), ('2');" +${CLICKHOUSE_CLIENT} --query="INSERT INTO test.fixed_string_test_table VALUES ('0'), ('2'), ('2');" +# 131072 -> 17 bit is 1 +${CLICKHOUSE_CLIENT} --query="INSERT INTO test.signed_integer_test_table VALUES (-2), (0), (2), (2), (131072), (131073), (131073);" +${CLICKHOUSE_CLIENT} --query="INSERT INTO test.unsigned_integer_test_table VALUES (0), (2), (2), (131072), (131073), (131073);" +${CLICKHOUSE_CLIENT} --query="INSERT INTO test.enum_test_table VALUES ('hello'), ('world'), ('world'), ('yandex'), ('clickhouse'), ('clickhouse');" +${CLICKHOUSE_CLIENT} --query="INSERT INTO test.date_test_table VALUES (1), (2), (2), (256), (257), (257);" + +export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} |sed 's/'"${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/debug/g'` + +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64" + +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int64" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt64" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> Int16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> UInt16" + +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int64" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt64" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> Int16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> UInt16" + + +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> Int8" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> UInt8" + + +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt32" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt16" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> Int8" +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM test.date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> UInt8" + +export CLICKHOUSE_CLIENT=`echo ${CLICKHOUSE_CLIENT} |sed 's/debug/'"${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g'` + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.string_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.fixed_string_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.signed_integer_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.unsigned_integer_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.enum_test_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.date_test_table;" + +${CLICKHOUSE_CLIENT} --query="SYSTEM START MERGES;" From 2ae6b6414c7131829f49adad610322693755ddd7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 17:16:07 +0300 Subject: [PATCH 112/181] Fixed error in 'compressor' tool: the 'level' parameter wasn't working #1345 --- dbms/programs/compressor/Compressor.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/programs/compressor/Compressor.cpp b/dbms/programs/compressor/Compressor.cpp index 4a412d987b4..544238bf581 100644 --- a/dbms/programs/compressor/Compressor.cpp +++ b/dbms/programs/compressor/Compressor.cpp @@ -61,7 +61,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("block-size,b", boost::program_options::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") - ("level", "compression level") + ("level", boost::program_options::value(), "compression level") ("none", "use no compression instead of LZ4") ("stat", "print block statistics of compressed data") ; @@ -94,7 +94,9 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) else if (use_none) method = DB::CompressionMethod::NONE; - DB::CompressionSettings settings(method, options.count("level") > 0 ? options["level"].as() : DB::CompressionSettings::getDefaultLevel(method)); + DB::CompressionSettings settings(method, options.count("level") + ? options["level"].as() + : DB::CompressionSettings::getDefaultLevel(method)); DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO); DB::WriteBufferFromFileDescriptor wb(STDOUT_FILENO); From 7b994d983ffc1e68aa22ca55fa49daea0a7fae87 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Dec 2018 17:26:54 +0300 Subject: [PATCH 113/181] Fix Native streams headers. --- dbms/programs/server/TCPHandler.cpp | 26 ++------------------------ 1 file changed, 2 insertions(+), 24 deletions(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 6f74b30ea02..2b2be14170d 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -370,19 +370,7 @@ void TCPHandler::processInsertQuery(const Settings & global_settings) } /// Send block to the client - table structure. - Block block = state.io.out->getHeader(); - - /// Support insert from old clients without low cardinality type. - if (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) - { - for (auto & col : block) - { - col.type = recursiveRemoveLowCardinality(col.type); - col.column = recursiveRemoveLowCardinality(col.column); - } - } - - sendData(block); + sendData(state.io.out->getHeader()); readData(global_settings); state.io.out->writeSuffix(); @@ -398,17 +386,7 @@ void TCPHandler::processOrdinaryQuery() /// Send header-block, to allow client to prepare output format for data to send. { Block header = state.io.in->getHeader(); - - /// Send data to old clients without low cardinality type. - if (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) - { - for (auto & column : header) - { - column.column = recursiveRemoveLowCardinality(column.column); - column.type = recursiveRemoveLowCardinality(column.type); - } - } - + if (header) sendData(header); } From fbcf6640c9c6312816fb0accbf7d9dd3dd1ba141 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Dec 2018 17:54:26 +0300 Subject: [PATCH 114/181] Fix style. --- dbms/programs/server/TCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 2b2be14170d..e4126b6dd03 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -386,7 +386,7 @@ void TCPHandler::processOrdinaryQuery() /// Send header-block, to allow client to prepare output format for data to send. { Block header = state.io.in->getHeader(); - + if (header) sendData(header); } From 731d76821d357683311cf84df1c080ece898a5d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 18:14:32 +0300 Subject: [PATCH 115/181] Merging abandoned pull request with "boundingRatio" function #3139 --- .../AggregateFunctionBoundingRatio.h | 133 +++++++++--------- .../00715_bounding_ratio.reference | 19 +++ .../0_stateless/00715_bounding_ratio.sql | 18 ++- 3 files changed, 99 insertions(+), 71 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 3cc6d92547b..f89943f1fc6 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -1,18 +1,10 @@ #pragma once -#include -#include -#include -#include -#include -#include #include +#include +#include #include #include -#include -#include -#include - #include #include @@ -20,61 +12,80 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +/** Tracks the leftmost and rightmost (x, y) data points. + */ struct AggregateFunctionBoundingRatioData { - using TimestampEvent = std::pair; - - bool is_first = false; - - TimestampEvent first_event; - TimestampEvent last_event; - - void add(UInt32 timestamp, Float64 f) + struct Point { - if (is_first) + Float64 x; + Float64 y; + }; + + bool empty = true; + Point left; + Point right; + + void add(Float64 x, Float64 y) + { + Point point{x, y}; + + if (empty) { - first_event = TimestampEvent{timestamp, f}; - is_first = true; + left = point; + right = point; + empty = false; } - else + else if (point.x < left.x) { - last_event = TimestampEvent{timestamp, f}; + left = point; + } + else if (point.x > right.x) + { + right = point; } } void merge(const AggregateFunctionBoundingRatioData & other) { - // if the arg is earlier than us, replace us with them - if (other.first_event.first < first_event.first) + if (empty) { - first_event = other.first_event; + *this = other; } - // if the arg is _later_ than us, replace us with them - if (other.last_event.first > last_event.second) + else { - last_event = other.last_event; + if (other.left.x < left.x) + left = other.left; + if (other.right.x > right.x) + right = other.right; } } void serialize(WriteBuffer & buf) const { - writeBinary(is_first, buf); - writeBinary(first_event.first, buf); - writeBinary(first_event.second, buf); + writeBinary(empty, buf); - writeBinary(last_event.first, buf); - writeBinary(last_event.second, buf); + if (!empty) + { + writePODBinary(left, buf); + writePODBinary(right, buf); + } } void deserialize(ReadBuffer & buf) { - readBinary(is_first, buf); + readBinary(empty, buf); - readBinary(first_event.first, buf); - readBinary(first_event.second, buf); - - readBinary(last_event.first, buf); - readBinary(last_event.second, buf); + if (!empty) + { + readPODBinary(left, buf); + readPODBinary(right, buf); + } } }; @@ -82,21 +93,15 @@ struct AggregateFunctionBoundingRatioData class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper { private: - /* implements a basic derivative function - * - * (y2 - y1) / (x2 - x1) - */ + /** Calculates the slope of a line between leftmost and rightmost data points. + * (y2 - y1) / (x2 - x1) + */ Float64 getBoundingRatio(const AggregateFunctionBoundingRatioData & data) const { - if (data.first_event.first == 0) - return 0; - if (data.last_event.first == 0) - return 0; - // void divide by zero in denominator - if (data.last_event.first == data.first_event.first) - return 0; + if (data.empty) + return std::numeric_limits::quiet_NaN(); - return (data.last_event.second - data.first_event.second) / (data.last_event.first - data.first_event.first); + return (data.right.y - data.left.y) / (data.right.x - data.left.x); } public: @@ -107,21 +112,14 @@ public: AggregateFunctionBoundingRatio(const DataTypes & arguments) { + const auto x_arg = arguments.at(0).get(); + const auto y_arg = arguments.at(0).get(); - - const auto time_arg = arguments.at(0).get(); - if (!typeid_cast(time_arg) && !typeid_cast(time_arg)) - throw Exception {"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName() - + ", must be DateTime or UInt32"}; - - const auto number_arg = arguments.at(1).get(); - if (!number_arg->isNumber()) - throw Exception {"Illegal type " + number_arg->getName() + " of argument " + toString(1) + " of aggregate function " + getName() - + ", must be a Number", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber()) + throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.", + ErrorCodes::BAD_ARGUMENTS); } - DataTypePtr getReturnType() const override { return std::make_shared(); @@ -129,9 +127,10 @@ public: void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override { - const auto timestamp = static_cast *>(columns[0])->getData()[row_num]; - const auto value = static_cast *>(columns[1])->getData()[row_num]; - data(place).add(timestamp, value); + /// TODO Inefficient. + const auto x = applyVisitor(FieldVisitorConvertToNumber(), (*columns[0])[row_num]); + const auto y = applyVisitor(FieldVisitorConvertToNumber(), (*columns[1])[row_num]); + data(place).add(x, y); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override diff --git a/dbms/tests/queries/0_stateless/00715_bounding_ratio.reference b/dbms/tests/queries/0_stateless/00715_bounding_ratio.reference index 6ed281c757a..f1e96af83a9 100644 --- a/dbms/tests/queries/0_stateless/00715_bounding_ratio.reference +++ b/dbms/tests/queries/0_stateless/00715_bounding_ratio.reference @@ -1,2 +1,21 @@ 1 1 +1.5 +1.5 +1.5 +0 1.5 +1 1.5 +2 1.5 +3 1.5 +4 1.5 +5 1.5 +6 1.5 +7 1.5 +8 1.5 +9 1.5 + +0 1.5 +1.5 +nan +nan +1 diff --git a/dbms/tests/queries/0_stateless/00715_bounding_ratio.sql b/dbms/tests/queries/0_stateless/00715_bounding_ratio.sql index 8cea70a09b1..ff3cd4c606b 100644 --- a/dbms/tests/queries/0_stateless/00715_bounding_ratio.sql +++ b/dbms/tests/queries/0_stateless/00715_bounding_ratio.sql @@ -3,14 +3,24 @@ drop table if exists rate_test; create table rate_test (timestamp UInt32, event UInt32) engine=Memory; insert into rate_test values (0,1000),(1,1001),(2,1002),(3,1003),(4,1004),(5,1005),(6,1006),(7,1007),(8,1008); -select 1.0 = rate(timestamp, event) from rate_test; +select 1.0 = boundingRatio(timestamp, event) from rate_test; drop table if exists rate_test2; create table rate_test2 (uid UInt32 default 1,timestamp DateTime, event UInt32) engine=Memory; -insert into rate_test2(timestamp, event) values ('2018-01-01 01:01:01',1001),('2018-01-01 01:01:02',1002),('2018-01-01 01:01:03',1003),('2018-01-01 01:01:04',1004),('2018-01-01 01:01:05',1005),('2018-01-01 01:01:06',1006),('2018-01-01 01:01:07',1007),('2018-01-01 01:01:08',1008); +insert into rate_test2(timestamp, event) values ('2018-01-01 01:01:01',1001),('2018-01-01 01:01:02',1002),('2018-01-01 01:01:03',1003),('2018-01-01 01:01:04',1004),('2018-01-01 01:01:05',1005),('2018-01-01 01:01:06',1006),('2018-01-01 01:01:07',1007),('2018-01-01 01:01:08',1008); - -select 1.0 = rate(timestamp, event ) from rate_test2; +select 1.0 = boundingRatio(timestamp, event) from rate_test2; drop table rate_test; drop table rate_test2; + + +SELECT boundingRatio(number, number * 1.5) FROM numbers(10); +SELECT boundingRatio(1000 + number, number * 1.5) FROM numbers(10); +SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(10); +SELECT number % 10 AS k, boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(100) GROUP BY k WITH TOTALS ORDER BY k; + +SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(2); +SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1); +SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1) WHERE 0; +SELECT boundingRatio(number, exp(number)) = e() - 1 FROM numbers(2); From 18b530544e848fc721d7d3d85a48dd94f2aa6112 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Dec 2018 18:17:07 +0300 Subject: [PATCH 116/181] Fix setting description. --- dbms/src/Interpreters/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index e6ac4c356eb..27a5ba29282 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -294,7 +294,7 @@ struct Settings M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \ M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.") \ M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.") \ - M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise convert LowCardinality columns to ordinary.") \ + M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.") \ #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; From 7fc7a4cb7872faa1a539c8ddf757dfa76dc6ba25 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 18:22:27 +0300 Subject: [PATCH 117/181] Suppress warning in protobuf build [#CLICKHOUSE-2] --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 66173322659..989761bfb67 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -2,7 +2,7 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-stringop-overflow") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -std=c++1z") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -Wno-sign-compare -std=c++1z") elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-format -Wno-parentheses-equality") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-format -std=c++1z") From ea018a0a36cafb0c010420adf50169fec832d6e7 Mon Sep 17 00:00:00 2001 From: ogorbacheva Date: Thu, 20 Dec 2018 18:33:47 +0300 Subject: [PATCH 118/181] Add the description of the CHECK TABLE query (#3881) * Added the description of the CHECK table query * some fixes * some fixes * fixes in CHECK TABLE doc --- docs/en/interfaces/formats.md | 2 +- docs/en/operations/settings/settings.md | 2 +- docs/en/query_language/misc.md | 45 ++++++++++++++++++++++--- 3 files changed, 42 insertions(+), 7 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 8ede8c5507d..a21e8b53d2a 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -169,7 +169,7 @@ When formatting, rows are enclosed in double quotes. A double quote inside a str clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -*By default, the delimiter is `,`. See the [format_csv_delimiter](/operations/settings/settings/#format_csv_delimiter) setting for more information. +*By default, the delimiter is `,`. See the [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) setting for more information. When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2d63c3e5e9a..3b4cf268579 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -149,7 +149,7 @@ Default value: 0 (off). Used when performing `SELECT` from a distributed table that points to replicated tables. -## max_threads +## max_threads {#max_threads} The maximum number of query processing threads diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index 148f4fe69f9..159a7611206 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -4,8 +4,8 @@ This query is exactly the same as `CREATE`, but -- instead of the word `CREATE` it uses the word `ATTACH`. -- The query doesn't create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table to the server. +- Instead of the word `CREATE` it uses the word `ATTACH`. +- The query does not create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table to the server. After executing an ATTACH query, the server will know about the existence of the table. If the table was previously detached (``DETACH``), meaning that its structure is known, you can use shorthand without defining the structure. @@ -16,6 +16,41 @@ ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] This query is used when starting the server. The server stores table metadata as files with `ATTACH` queries, which it simply runs at launch (with the exception of system tables, which are explicitly created on the server). +## CHECK TABLE + +Checks if the data in the table is corrupted. + +``` sql +CHECK TABLE [db.]name +``` + +The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. + +The query response contains the `result` column with a single row. The row has a value of + [Boolean](../data_types/boolean.md) type: + +- 0 - The data in the table is corrupted. +- 1 - The data maintains integrity. + +The `CHECK TABLE` query is only supported for the following table engines: + +- [Log](../operations/table_engines/log.md) +- [TinyLog](../operations/table_engines/tinylog.md) +- StripeLog + +These engines do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner. + +To avoid data loss use the [MergeTree](../operations/table_engines/mergetree.md) family tables. + +**If the data is corrupted** + +If the table is corrupted, you can copy the non-corrupted data to another table. To do this: + +1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE AS `. +2. Set the [max_threads](../operations/settings/settings.md#max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. +3. Execute the query `INSERT INTO SELECT * FROM `. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied. +4. Restart the `clickhouse-client` to reset the `max_threads` value. + ## DESCRIBE TABLE ``` sql @@ -198,8 +233,8 @@ SHOW [TEMPORARY] TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORM Displays a list of tables -- tables from the current database, or from the 'db' database if "FROM db" is specified. -- all tables, or tables whose name matches the pattern, if "LIKE 'pattern'" is specified. +- Tables from the current database, or from the 'db' database if "FROM db" is specified. +- All tables, or tables whose name matches the pattern, if "LIKE 'pattern'" is specified. This query is identical to: `SELECT name FROM system.tables WHERE database = 'db' [AND name LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format]`. @@ -207,7 +242,7 @@ See also the section "LIKE operator". ## TRUNCATE -```sql +``` sql TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` From c8cab3cf1ebefc2a0344ff77799129dc0dcadcf2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 19:11:22 +0300 Subject: [PATCH 119/181] Added performance test #3139 --- .../performance/no_data/bounding_ratio.xml | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 dbms/tests/performance/no_data/bounding_ratio.xml diff --git a/dbms/tests/performance/no_data/bounding_ratio.xml b/dbms/tests/performance/no_data/bounding_ratio.xml new file mode 100644 index 00000000000..269a7e21e51 --- /dev/null +++ b/dbms/tests/performance/no_data/bounding_ratio.xml @@ -0,0 +1,19 @@ + + bounding_ratio + once + + + + + 1000 + 10000 + + + + + + + + SELECT boundingRatio(number, number) FROM system.numbers + SELECT (argMax(number, number) - argMin(number, number)) / (max(number) - min(number)) FROM system.numbers + From 7f544da83cc7d12fda28f21d665fb1b05e73a7f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 19:11:22 +0300 Subject: [PATCH 120/181] Added performance test #3139 --- .../performance/no_data/bounding_ratio.xml | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 dbms/tests/performance/no_data/bounding_ratio.xml diff --git a/dbms/tests/performance/no_data/bounding_ratio.xml b/dbms/tests/performance/no_data/bounding_ratio.xml new file mode 100644 index 00000000000..269a7e21e51 --- /dev/null +++ b/dbms/tests/performance/no_data/bounding_ratio.xml @@ -0,0 +1,19 @@ + + bounding_ratio + once + + + + + 1000 + 10000 + + + + + + + + SELECT boundingRatio(number, number) FROM system.numbers + SELECT (argMax(number, number) - argMin(number, number)) / (max(number) - min(number)) FROM system.numbers + From edc415857c87fec4f789e5d7611ec87c67cb479b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 20:46:31 +0300 Subject: [PATCH 121/181] Removed accidential change [#CLICKHOUSE-2] --- dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp index bf46bccd9c7..52090e9cac5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp @@ -23,7 +23,7 @@ namespace AggregateFunctionPtr createAggregateFunctionHistogram(const std::string &, const DataTypes &, const Array &) { -/* if (params.size() != 1) + if (params.size() != 1) throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (params[0].getType() != Field::Types::UInt64) @@ -44,7 +44,7 @@ AggregateFunctionPtr createAggregateFunctionHistogram(const std::string &, const if (!res) throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return res;*/ + return res; return nullptr; } From b7b418a6f53d67c100f7f92d1cc2e34c5d54e898 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 20:48:11 +0300 Subject: [PATCH 122/181] Renamed setting #3851 --- dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Storages/StorageDistributed.cpp | 2 +- ...d_optimize_skip_select_on_unused_shards.sh | 32 +++++++++---------- 3 files changed, 18 insertions(+), 18 deletions(-) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index c325cc2f845..f2f24e92017 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -89,7 +89,7 @@ struct Settings M(SettingBool, skip_unavailable_shards, false, "Silently skip unavailable shards.") \ \ M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ - M(SettingBool, distributed_optimize_skip_select_on_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ + M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.") \ M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.") \ diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 0429db5cef6..9086d1bf321 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -294,7 +294,7 @@ BlockInputStreams StorageDistributed::read( : ClusterProxy::SelectStreamFactory( header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables()); - if (settings.distributed_optimize_skip_select_on_unused_shards) + if (settings.optimize_skip_unused_shards) { auto smaller_cluster = skipUnusedShards(cluster, query_info); diff --git a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh index 92af8677058..6adcec4a14e 100755 --- a/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh +++ b/dbms/tests/queries/0_stateless/00754_distributed_optimize_skip_select_on_unused_shards.sh @@ -18,88 +18,88 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO test.mergetree VALUES (1, 1, 1);" ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM test.distributed;" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' -# Should fail without setting `distributed_optimize_skip_select_on_unused_shards` +# Should fail without setting `optimize_skip_unused_shards` ${CLICKHOUSE_CLIENT} --query "SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0;" 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Should pass now ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0; " # Should still fail because of matching unavailable shard ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 2 AND b = 2; " 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' # Try more complext expressions for constant folding - all should pass. ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 1 AND a = 0 AND b = 0; " ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a IN (0, 1) AND b IN (0, 1); " ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 OR a = 1 AND b = 1; " # TODO: should pass one day. #${CLICKHOUSE_CLIENT} -n --query=" -# SET distributed_optimize_skip_select_on_unused_shards = 1; +# SET optimize_skip_unused_shards = 1; # SELECT count(*) FROM test.distributed WHERE a = 0 AND b >= 0 AND b <= 1; #" ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 AND c = 0; " ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 AND c != 10; " ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 AND (a+b)*b != 12; " ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE (a = 0 OR a = 1) AND (b = 0 OR b = 1); " # These ones should fail. ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b <= 1; " 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND c = 0; " 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 OR a = 1 AND b = 0; " 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 OR a = 2 AND b = 2; " 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} -n --query=" - SET distributed_optimize_skip_select_on_unused_shards = 1; + SET optimize_skip_unused_shards = 1; SELECT count(*) FROM test.distributed WHERE a = 0 AND b = 0 OR c = 0; " 2>&1 \ | fgrep -q "All connection tries failed" && echo 'OK' || echo 'FAIL' From 1841634ae253ba230a8a998e3f0a92d01cd919ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 20:49:37 +0300 Subject: [PATCH 123/181] Removed accidential change [#CLICKHOUSE-2] --- dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp index 52090e9cac5..eaacb10be01 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes namespace { -AggregateFunctionPtr createAggregateFunctionHistogram(const std::string &, const DataTypes &, const Array &) +AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params) { if (params.size() != 1) throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); From 980d6d9e46e6a90bbe7a13af6d0209b40856ac77 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 21:16:06 +0300 Subject: [PATCH 124/181] Fixed error in "regexpQuoteMeta" function #3826 --- dbms/src/Functions/regexpQuoteMeta.cpp | 17 ++++++++--------- dbms/src/Functions/trim.cpp | 2 +- dbms/src/IO/WriteBuffer.h | 2 +- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/dbms/src/Functions/regexpQuoteMeta.cpp b/dbms/src/Functions/regexpQuoteMeta.cpp index b14a58ba366..720e2029343 100644 --- a/dbms/src/Functions/regexpQuoteMeta.cpp +++ b/dbms/src/Functions/regexpQuoteMeta.cpp @@ -76,7 +76,6 @@ public: auto & dst_data = dst_column->getChars(); auto & dst_offsets = dst_column->getOffsets(); - dst_data.resize(input->getChars().size() * input->size()); dst_offsets.resize(input_rows_count); const ColumnString::Offsets & src_offsets = input->getOffsets(); @@ -86,22 +85,22 @@ public: auto dst_pos = dst; size_t src_offset_prev = 0; - for (size_t row = 0; row < input_rows_count; ++row) { size_t srclen = src_offsets[row] - src_offset_prev - 1; /// suboptimal, but uses original implementation from re2 - re2_st::StringPiece unquoted(source, srclen); - const auto & quoted = re2_st::RE2::QuoteMeta(unquoted); - const auto size = quoted.size(); - std::memcpy(dst_pos, quoted.data(), size); + const auto & quoted = re2_st::RE2::QuoteMeta({source, srclen}); + const auto size = quoted.size() + 1; + + size_t new_dst_size = dst_data.size() + size; + dst_data.resize(new_dst_size); + memcpy(dst_pos, quoted.c_str(), size); source += srclen + 1; - dst_pos[size] = '\0'; - dst_pos += size + 1; + dst_pos += size; - dst_offsets[row] = dst_pos - dst; + dst_offsets[row] = new_dst_size; src_offset_prev = src_offsets[row]; } diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index 42aa59bffe5..10c9b6557aa 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -60,7 +60,7 @@ public: execute(reinterpret_cast(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length); res_data.resize(res_data.size() + length + 1); - std::memcpy(&res_data[res_offset], start, length); + memcpy(&res_data[res_offset], start, length); res_offset += length + 1; res_data[res_offset - 1] = '\0'; diff --git a/dbms/src/IO/WriteBuffer.h b/dbms/src/IO/WriteBuffer.h index 1d25d721ed0..e6a73ea90e3 100644 --- a/dbms/src/IO/WriteBuffer.h +++ b/dbms/src/IO/WriteBuffer.h @@ -76,7 +76,7 @@ public: { nextIfAtEnd(); size_t bytes_to_copy = std::min(static_cast(working_buffer.end() - pos), n - bytes_copied); - std::memcpy(pos, from + bytes_copied, bytes_to_copy); + memcpy(pos, from + bytes_copied, bytes_to_copy); pos += bytes_to_copy; bytes_copied += bytes_to_copy; } From b7afe46dfdb7c1f47801ecd19439f1188ad226be Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 20 Dec 2018 21:57:20 +0300 Subject: [PATCH 125/181] remove ALTER MODIFY PRIMARY KEY command [#CLICKHOUSE-4214] --- dbms/src/Interpreters/DDLWorker.cpp | 1 - dbms/src/Parsers/ASTAlterQuery.cpp | 10 -- dbms/src/Parsers/ASTAlterQuery.h | 5 - dbms/src/Parsers/ParserAlterQuery.cpp | 14 +- dbms/src/Storages/AlterCommands.cpp | 14 -- dbms/src/Storages/AlterCommands.h | 6 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 64 --------- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 - .../ReplicatedMergeTreeAlterThread.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 33 +---- .../Storages/StorageReplicatedMergeTree.cpp | 8 +- .../00329_alter_primary_key.reference | 130 ------------------ .../0_stateless/00329_alter_primary_key.sql | 83 ----------- 13 files changed, 10 insertions(+), 362 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/00329_alter_primary_key.reference delete mode 100644 dbms/tests/queries/0_stateless/00329_alter_primary_key.sql diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 60183cd0c3a..808ff9b1cf9 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -204,7 +204,6 @@ static bool isSupportedAlterType(int type) ASTAlterCommand::ADD_COLUMN, ASTAlterCommand::DROP_COLUMN, ASTAlterCommand::MODIFY_COLUMN, - ASTAlterCommand::MODIFY_PRIMARY_KEY, ASTAlterCommand::DROP_PARTITION, ASTAlterCommand::DELETE, ASTAlterCommand::UPDATE, diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 3577346df0f..feec84d6e98 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -25,11 +25,6 @@ ASTPtr ASTAlterCommand::clone() const res->column = column->clone(); res->children.push_back(res->column); } - if (primary_key) - { - res->primary_key = primary_key->clone(); - res->children.push_back(res->primary_key); - } if (order_by) { res->order_by = order_by->clone(); @@ -82,11 +77,6 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : ""); col_decl->formatImpl(settings, state, frame); } - else if (type == ASTAlterCommand::MODIFY_PRIMARY_KEY) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY PRIMARY KEY " << (settings.hilite ? hilite_none : ""); - primary_key->formatImpl(settings, state, frame); - } else if (type == ASTAlterCommand::MODIFY_ORDER_BY) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index b73e1f38e2c..e1cd74cb4b5 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -26,7 +26,6 @@ public: DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN, - MODIFY_PRIMARY_KEY, MODIFY_ORDER_BY, DROP_PARTITION, @@ -55,10 +54,6 @@ public: */ ASTPtr column; - /** For MODIFY PRIMARY KEY - */ - ASTPtr primary_key; - /** For MODIFY ORDER BY */ ASTPtr order_by; diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 859c9b6af51..83ad42ebbcb 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -24,7 +24,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_clear_column("CLEAR COLUMN"); ParserKeyword s_modify_column("MODIFY COLUMN"); ParserKeyword s_comment_column("COMMENT COLUMN"); - ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY"); ParserKeyword s_modify_order_by("MODIFY ORDER BY"); ParserKeyword s_attach_partition("ATTACH PARTITION"); @@ -196,13 +195,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::MODIFY_COLUMN; } - else if (s_modify_primary_key.ignore(pos, expected)) - { - if (!parser_exp_elem.parse(pos, command->primary_key, expected)) - return false; - - command->type = ASTAlterCommand::MODIFY_PRIMARY_KEY; - } else if (s_modify_order_by.ignore(pos, expected)) { if (!parser_exp_elem.parse(pos, command->order_by, expected)) @@ -247,14 +239,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->children.push_back(command->col_decl); if (command->column) command->children.push_back(command->column); - if (command->primary_key) - command->children.push_back(command->primary_key); if (command->partition) command->children.push_back(command->partition); + if (command->order_by) + command->children.push_back(command->order_by); if (command->predicate) command->children.push_back(command->predicate); if (command->update_assignments) command->children.push_back(command->update_assignments); + if (command->comment) + command->children.push_back(command->comment); return true; } diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 37ac0ed64a6..ba5ed9e21bd 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -101,13 +101,6 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.comment = ast_comment.value.get(); return command; } - else if (command_ast->type == ASTAlterCommand::MODIFY_PRIMARY_KEY) - { - AlterCommand command; - command.type = AlterCommand::MODIFY_PRIMARY_KEY; - command.primary_key = command_ast->primary_key; - return command; - } else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY) { AlterCommand command; @@ -271,13 +264,6 @@ void AlterCommand::apply(ColumnsDescription & columns_description, ASTPtr & orde /// both old and new columns have default expression, update it columns_description.defaults[column_name].expression = default_expression; } - else if (type == MODIFY_PRIMARY_KEY) - { - if (!primary_key_ast) - order_by_ast = primary_key; - else - primary_key_ast = primary_key; - } else if (type == MODIFY_ORDER_BY) { if (!primary_key_ast) diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index af606aa84ef..f1adbdaf9b0 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -22,7 +22,6 @@ struct AlterCommand DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN, - MODIFY_PRIMARY_KEY, MODIFY_ORDER_BY, UKNOWN_TYPE, }; @@ -44,9 +43,6 @@ struct AlterCommand /// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible. String after_column; - /// For MODIFY_PRIMARY_KEY - ASTPtr primary_key; - /// For MODIFY_ORDER_BY ASTPtr order_by; @@ -73,7 +69,7 @@ class AlterCommands : public std::vector public: void apply(ColumnsDescription & columns_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast) const; - /// For storages that don't support MODIFY_PRIMARY_KEY or MODIFY_ORDER_BY. + /// For storages that don't support MODIFY_ORDER_BY. void apply(ColumnsDescription & columns_description) const; void validate(const IStorage & table, const Context & context); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index fd2d9d9d50d..b65d23f47e1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1229,7 +1229,6 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( const DataPartPtr & part, const NamesAndTypesList & new_columns, - const ASTPtr & new_primary_key_expr_list, bool skip_sanity_checks) { ExpressionActionsPtr expression; @@ -1290,63 +1289,6 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( DataPart::Checksums add_checksums; - /// Update primary key if needed. - size_t new_primary_key_file_size{}; - MergeTreeDataPartChecksum::uint128 new_primary_key_hash{}; - - if (new_primary_key_expr_list) - { - ASTPtr query = new_primary_key_expr_list; - auto syntax_result = SyntaxAnalyzer(context, {}).analyze(query, new_columns); - ExpressionActionsPtr new_primary_expr = ExpressionAnalyzer(query, syntax_result, context).getActions(true); - Block new_primary_key_sample = new_primary_expr->getSampleBlock(); - size_t new_key_size = new_primary_key_sample.columns(); - - Columns new_index(new_key_size); - - /// Copy the existing primary key columns. Fill new columns with default values. - /// NOTE default expressions are not supported. - - ssize_t prev_position_of_existing_column = -1; - for (size_t i = 0; i < new_key_size; ++i) - { - const String & column_name = new_primary_key_sample.safeGetByPosition(i).name; - - if (primary_key_sample.has(column_name)) - { - ssize_t position_of_existing_column = primary_key_sample.getPositionByName(column_name); - - if (position_of_existing_column < prev_position_of_existing_column) - throw Exception("Permuting of columns of primary key is not supported", ErrorCodes::BAD_ARGUMENTS); - - new_index[i] = part->index.at(position_of_existing_column); - prev_position_of_existing_column = position_of_existing_column; - } - else - { - const IDataType & type = *new_primary_key_sample.safeGetByPosition(i).type; - new_index[i] = type.createColumnConstWithDefaultValue(part->marks_count)->convertToFullColumnIfConst(); - } - } - - if (prev_position_of_existing_column == -1) - throw Exception("No common columns while modifying primary key", ErrorCodes::BAD_ARGUMENTS); - - String index_tmp_path = full_path + part->name + "/primary.idx.tmp"; - WriteBufferFromFile index_file(index_tmp_path); - HashingWriteBuffer index_stream(index_file); - - for (size_t i = 0, marks_count = part->marks_count; i < marks_count; ++i) - for (size_t j = 0; j < new_key_size; ++j) - new_primary_key_sample.getByPosition(j).type->serializeBinary(*new_index[j].get(), i, index_stream); - - transaction->rename_map["primary.idx.tmp"] = "primary.idx"; - - index_stream.next(); - new_primary_key_file_size = index_stream.count(); - new_primary_key_hash = index_stream.getHash(); - } - if (transaction->rename_map.empty() && !force_update_metadata) { transaction->clear(); @@ -1395,12 +1337,6 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( new_checksums.files[it.second] = add_checksums.files[it.first]; } - if (new_primary_key_file_size) - { - new_checksums.files["primary.idx"].file_size = new_primary_key_file_size; - new_checksums.files["primary.idx"].file_hash = new_primary_key_hash; - } - /// Write the checksums to the temporary file. if (!part->checksums.empty()) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 4670f8b9560..b8f01c40077 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -479,13 +479,11 @@ public: /// Performs ALTER of the data part, writes the result to temporary files. /// Returns an object allowing to rename temporary files to permanent files. - /// If new_primary_key_expr_list is not nullptr, will prepare the new primary.idx file. /// If the number of affected columns is suspiciously high and skip_sanity_checks is false, throws an exception. /// If no data transformations are necessary, returns nullptr. AlterDataPartTransactionPtr alterDataPart( const DataPartPtr & part, const NamesAndTypesList & new_columns, - const ASTPtr & new_primary_key_expr_list, bool skip_sanity_checks); /// Freezes all parts. diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index 6f30e27bc4f..e7fe9dba256 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -150,7 +150,7 @@ void ReplicatedMergeTreeAlterThread::run() /// Update the part and write result to temporary files. /// TODO: You can skip checking for too large changes if ZooKeeper has, for example, /// node /flags/force_alter. - auto transaction = storage.data.alterDataPart(part, columns_for_parts, nullptr, false); + auto transaction = storage.data.alterDataPart(part, columns_for_parts, false); if (!transaction) continue; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index f4f69e3ac87..6ee1e7ca9c9 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -210,24 +210,12 @@ void StorageMergeTree::alter( ASTPtr new_primary_key_ast = data.primary_key_ast; params.apply(new_columns, new_order_by_ast, new_primary_key_ast); - ASTPtr primary_expr_list_for_altering_parts; - for (const AlterCommand & param : params) - { - if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) - { - if (supportsSampling()) - throw Exception("MODIFY PRIMARY KEY only supported for tables without sampling key", ErrorCodes::BAD_ARGUMENTS); - - primary_expr_list_for_altering_parts = MergeTreeData::extractKeyExpressionList(param.primary_key); - } - } - auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); auto columns_for_parts = new_columns.getAllPhysical(); std::vector transactions; for (const MergeTreeData::DataPartPtr & part : parts) { - if (auto transaction = data.alterDataPart(part, columns_for_parts, primary_expr_list_for_altering_parts, false)) + if (auto transaction = data.alterDataPart(part, columns_for_parts, false)) transactions.push_back(std::move(transaction)); } @@ -238,19 +226,7 @@ void StorageMergeTree::alter( auto & storage_ast = typeid_cast(ast); if (new_order_by_ast.get() != data.order_by_ast.get()) - { - if (storage_ast.order_by) - { - /// The table was created using the "new" syntax (with key expressions in separate clauses). - storage_ast.set(storage_ast.order_by, new_order_by_ast); - } - else - { - /// Primary key is in the second place in table engine description and can be represented as a tuple. - /// TODO: Not always in second place. If there is a sampling key, then the third one. Fix it. - storage_ast.engine->arguments->children.at(1) = new_order_by_ast; - } - } + storage_ast.set(storage_ast.order_by, new_order_by_ast); if (new_primary_key_ast.get() != data.primary_key_ast.get()) storage_ast.set(storage_ast.primary_key, new_primary_key_ast); @@ -266,9 +242,6 @@ void StorageMergeTree::alter( /// Columns sizes could be changed data.recalculateColumnSizes(); - - if (primary_expr_list_for_altering_parts) - data.loadDataParts(false); } @@ -725,7 +698,7 @@ void StorageMergeTree::clearColumnInPartition(const ASTPtr & partition, const Fi if (part->info.partition_id != partition_id) throw Exception("Unexpected partition ID " + part->info.partition_id + ". This is a bug.", ErrorCodes::LOGICAL_ERROR); - if (auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false)) + if (auto transaction = data.alterDataPart(part, columns_for_parts, false)) transactions.push_back(std::move(transaction)); LOG_DEBUG(log, "Removing column " << get(column_name) << " from part " << part->name); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 10981823b66..afe8cbc02ab 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1504,7 +1504,7 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name); - auto transaction = data.alterDataPart(part, columns_for_parts, nullptr, false); + auto transaction = data.alterDataPart(part, columns_for_parts, false); if (!transaction) continue; @@ -3059,12 +3059,6 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, data.checkAlter(params); - for (const AlterCommand & param : params) - { - if (param.type == AlterCommand::MODIFY_PRIMARY_KEY) - throw Exception("Modification of primary key is not supported for replicated tables", ErrorCodes::NOT_IMPLEMENTED); - } - ColumnsDescription new_columns = data.getColumns(); ASTPtr new_order_by_ast = data.order_by_ast; ASTPtr new_primary_key_ast = data.primary_key_ast; diff --git a/dbms/tests/queries/0_stateless/00329_alter_primary_key.reference b/dbms/tests/queries/0_stateless/00329_alter_primary_key.reference deleted file mode 100644 index 2b32543ce55..00000000000 --- a/dbms/tests/queries/0_stateless/00329_alter_primary_key.reference +++ /dev/null @@ -1,130 +0,0 @@ -1 -2 -3 -2 -3 -1 -2 -3 -2 -3 -1 -2 -3 -2 -3 -2 -3 -1 -1 Hello -2 -2 World -3 -3 abc -4 def -2 -2 World -3 -3 abc -4 def -2 World -3 abc -4 def -2 -2 World -3 -3 abc -4 def -1 -2 -3 -1 -1 Hello -2 -2 World -3 -3 abc -4 def -2 -2 World -3 -3 abc -4 def -2 World -3 abc -4 def -2 -2 World -3 -3 abc -4 def -1 -2 -3 -1 -1 Hello -2 -2 World -3 -3 abc -4 def -1 -1 Hello -2 -2 World -3 -3 abc -4 def -1 -1 Hello -2 -2 World -3 -3 abc -4 def -2 -2 World -3 -3 abc -4 def -2 World -3 abc -4 def -2 -2 World -3 -3 abc -4 def -1 -2 -3 -1 -1 Hello -2 -2 World -3 -3 abc -4 def -2 -2 World -3 -3 abc -4 def -2 World -3 abc -4 def -2 -2 World -3 -3 abc -4 def -1 -2 -3 -*** Check table creation statement *** -CREATE TABLE test.pk2 ( x UInt32, y UInt32, z UInt32) ENGINE = MergeTree PRIMARY KEY (x, y) ORDER BY (x, y, z) SETTINGS index_granularity = 8192 -*** Check that the inserted values were correctly sorted *** -100 20 1 -100 20 2 -100 30 1 -100 30 2 diff --git a/dbms/tests/queries/0_stateless/00329_alter_primary_key.sql b/dbms/tests/queries/0_stateless/00329_alter_primary_key.sql deleted file mode 100644 index 0d0ad6d2f96..00000000000 --- a/dbms/tests/queries/0_stateless/00329_alter_primary_key.sql +++ /dev/null @@ -1,83 +0,0 @@ -SET send_logs_level = 'none'; - -DROP TABLE IF EXISTS test.pk; -CREATE TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 1); - -INSERT INTO test.pk (x) VALUES (1), (2), (3); - -SELECT x FROM test.pk ORDER BY x; -SELECT x FROM test.pk WHERE x >= 2 ORDER BY x; - -ALTER TABLE test.pk MODIFY PRIMARY KEY (x); - -SELECT x FROM test.pk ORDER BY x; -SELECT x FROM test.pk WHERE x >= 2 ORDER BY x; - -ALTER TABLE test.pk ADD COLUMN y String, MODIFY PRIMARY KEY (x, y); - -SELECT x, y FROM test.pk ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y = '' ORDER BY x, y; - -INSERT INTO test.pk (x, y) VALUES (1, 'Hello'), (2, 'World'), (3, 'abc'), (4, 'def'); - -SELECT x, y FROM test.pk ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y; - -DETACH TABLE test.pk; -ATTACH TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64, y String) ENGINE = MergeTree(d, (x, y), 1); - -SELECT x, y FROM test.pk ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y; - -SET max_rows_to_read = 3; -SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y; -SET max_rows_to_read = 0; - -OPTIMIZE TABLE test.pk; -SELECT x, y FROM test.pk; -SELECT x, y FROM test.pk ORDER BY x, y; - -ALTER TABLE test.pk MODIFY PRIMARY KEY (x); - -SELECT x, y FROM test.pk ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y; - -DETACH TABLE test.pk; -ATTACH TABLE test.pk (d Date DEFAULT '2000-01-01', x UInt64, y String) ENGINE = MergeTree(d, (x), 1); - -SELECT x, y FROM test.pk ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y > '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x >= 2 AND y >= '' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE x > 2 AND y > 'z' ORDER BY x, y; -SELECT x, y FROM test.pk WHERE y < 'A' ORDER BY x, y; - -DROP TABLE test.pk; - -DROP TABLE IF EXISTS test.pk2; -CREATE TABLE test.pk2 (x UInt32) ENGINE MergeTree ORDER BY x; - -ALTER TABLE test.pk2 ADD COLUMN y UInt32, ADD COLUMN z UInt32, MODIFY ORDER BY (x, y, z); -ALTER TABLE test.pk2 MODIFY PRIMARY KEY (y); -- { serverError 36 } -ALTER TABLE test.pk2 MODIFY PRIMARY KEY (x, y); -SELECT '*** Check table creation statement ***'; -SHOW CREATE TABLE test.pk2; - -INSERT INTO test.pk2 VALUES (100, 30, 2), (100, 30, 1), (100, 20, 2), (100, 20, 1); -SELECT '*** Check that the inserted values were correctly sorted ***'; -SELECT * FROM test.pk2; - -DROP TABLE test.pk2; From 34e8a0bf1914bcd3fd836e269a113f011def9ba1 Mon Sep 17 00:00:00 2001 From: Amy Krishnevsky Date: Thu, 20 Dec 2018 22:02:52 +0300 Subject: [PATCH 126/181] translated changelog --- CHANGELOG.md | 95 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 95 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 00ae12339b2..6058d40bab5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,98 @@ +## ClickHouse release 18.16.0, 2018-12-14 + +### New features: + +* `DEFAULT` expressions are evaluated for missing fields when loading data in semi-structured input formats (`JSONEachRow`, `TSKV`). [#3555](https://github.com/yandex/ClickHouse/pull/3555) +* The `ALTER TABLE` query now has the `MODIFY ORDER BY` action for changing the sorting key when adding or removing a table column. This is useful for tables in the `MergeTree` family that perform additional tasks when merging based on this sorting key, such as `SummingMergeTree`, `AggregatingMergeTree`, and so on. [#3581](https://github.com/yandex/ClickHouse/pull/3581) [#3755](https://github.com/yandex/ClickHouse/pull/3755) +* For tables in the `MergeTree` family, now you can specify a different sorting key (`ORDER BY`) and index (`PRIMARY KEY`). The sorting key can be longer than the index. [#3581](https://github.com/yandex/ClickHouse/pull/3581) +* Added the `hdfs` table function and the `HDFS` table engine for importing and exporting data to HDFS. [chenxing-xc](https://github.com/yandex/ClickHouse/pull/3617) +* Added functions for working with base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. [Alexander Krasheninnikov](https://github.com/yandex/ClickHouse/pull/3350) +* Now you can use a parameter to configure the precision of the `uniqCombined` aggregate function (select the number of HyperLogLog cells). [#3406](https://github.com/yandex/ClickHouse/pull/3406) +* Added the `system.contributors` table that contains the names of everyone who made commits in ClickHouse. [#3452](https://github.com/yandex/ClickHouse/pull/3452) +* Added the ability to omit the partition for the `ALTER TABLE ... FREEZE` request in order to back up all partitions at once. [#3514](https://github.com/yandex/ClickHouse/pull/3514) +* Added `dictGet` and `dictGetOrDefault` functions that don't require specifying the type of return value. The type is determined automatically from the dictionary description. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3564) +* Now you can specify comments for a column in the table description and change it using `ALTER`. [#3377](https://github.com/yandex/ClickHouse/pull/3377) +* Reading is supported for `Join` type tables with simple keys. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3728) +* Now you can specify the options `join_use_nulls`, `max_rows_in_join`, `max_bytes_in_join`, and `join_overflow_mode` when creating a `Join` type table. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3728) +* Added the `joinGet` function that allows you to use a `Join` type table like a dictionary. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3728) +* Added the `partition_key`, `sorting_key`, `primary_key`, and `sampling_key` columns to the `system.tables` table in order to provide information about table keys. [#3609](https://github.com/yandex/ClickHouse/pull/3609) +* Added the `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, and `is_in_sampling_key` columns to the `system.columns` table. [#3609](https://github.com/yandex/ClickHouse/pull/3609) +* Added the `min_time` and `max_time` columns to the `system.parts` table. These columns are populated when the partitioning key is an expression consisting of `DateTime` columns. [Emmanuel Donin de Rosière](https://github.com/yandex/ClickHouse/pull/3800) + +### Bug fixes: + +* Fixes and performance improvements for the `LowCardinality` data type. `GROUP BY` using `LowCardinality(Nullable(...))`. Getting the values of `extremes`. Processing high-order functions. `LEFT JOIN ARRAY`. Distributed `GROUP BY`. Functions that return `Array`. Execution of `ORDER BY`. Writing to `Distributed` tables (nicelulu). Backward compatibility for `INSERT` queries from old clients that implement the `Native` protocol. Support for `LowCardinality` for `JOIN`. Improved performance when working in a single stream. [#3823](https://github.com/yandex/ClickHouse/pull/3823) [#3803](https://github.com/yandex/ClickHouse/pull/3803) [#3799](https://github.com/yandex/ClickHouse/pull/3799) [#3769](https://github.com/yandex/ClickHouse/pull/3769) [#3744](https://github.com/yandex/ClickHouse/pull/3744) [#3681](https://github.com/yandex/ClickHouse/pull/3681) [#3651](https://github.com/yandex/ClickHouse/pull/3651) [#3649](https://github.com/yandex/ClickHouse/pull/3649) [#3641](https://github.com/yandex/ClickHouse/pull/3641) [#3632](https://github.com/yandex/ClickHouse/pull/3632) [#3568](https://github.com/yandex/ClickHouse/pull/3568) [#3523](https://github.com/yandex/ClickHouse/pull/3523) [#3518](https://github.com/yandex/ClickHouse/pull/3518) +* Fixed how the `select_sequential_consistency` option works. Previously, when this setting was enabled, an incomplete result was sometimes returned after beginning to write to a new partition. [#2863](https://github.com/yandex/ClickHouse/pull/2863) +* Databases are correctly specified when executing DDL `ON CLUSTER` queries and `ALTER UPDATE/DELETE`. [#3772](https://github.com/yandex/ClickHouse/pull/3772) [#3460](https://github.com/yandex/ClickHouse/pull/3460) +* Databases are correctly specified for subqueries inside a VIEW. [#3521](https://github.com/yandex/ClickHouse/pull/3521) +* Fixed a bug in `PREWHERE` with `FINAL` for `VersionedCollapsingMergeTree`. [7167bfd7](https://github.com/yandex/ClickHouse/commit/7167bfd7b365538f7a91c4307ad77e552ab4e8c1) +* Now you can use `KILL QUERY` to cancel queries that have not started yet because they are waiting for the table to be locked. [#3517](https://github.com/yandex/ClickHouse/pull/3517) +* Corrected date and time calculations if the clocks were moved back at midnight (this happens in Iran, and happened in Moscow from 1981 to 1983). Previously, this led to the time being reset a day earlier than necessary, and also caused incorrect formatting of the date and time in text format. [#3819](https://github.com/yandex/ClickHouse/pull/3819) +* Fixed bugs in some cases of `VIEW` and subqueries that omit the database. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/3521) +* Fixed a race condition when simultaneously reading from a `MATERIALIZED VIEW` and deleting a `MATERIALIZED VIEW` due to not locking the internal `MATERIALIZED VIEW`. [#3404](https://github.com/yandex/ClickHouse/pull/3404) [#3694](https://github.com/yandex/ClickHouse/pull/3694) +* Fixed the error `Lock handler cannot be nullptr.` [#3689](https://github.com/yandex/ClickHouse/pull/3689) +* Fixed query processing when the `compile_expressions` option is enabled (it's enabled by default). Nondeterministic constant expressions like the `now` function are no longer unfolded. [#3457](https://github.com/yandex/ClickHouse/pull/3457) +* Fixed a crash when specifying a non-constant scale argument in `toDecimal32/64/128` functions. +* Fixed an error when trying to insert an array with `NULL` elements in the `Values` format into a column of type `Array` without `Nullable` (if `input_format_values_interpret_expressions` = 1). [#3487](https://github.com/yandex/ClickHouse/pull/3487) [#3503](https://github.com/yandex/ClickHouse/pull/3503) +* Fixed continuous error logging in `DDLWorker` if ZooKeeper is not available. [8f50c620](https://github.com/yandex/ClickHouse/commit/8f50c620334988b28018213ec0092fe6423847e2) +* Fixed the return type for `quantile*` functions from `Date` and `DateTime` types of arguments. [#3580](https://github.com/yandex/ClickHouse/pull/3580) +* Fixed the `WITH` clause if it specifies a simple alias without expressions. [#3570](https://github.com/yandex/ClickHouse/pull/3570) +* Fixed processing of queries with named sub-queries and qualified column names when `enable_optimize_predicate_expression` is enabled. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/3588) +* Fixed the error `Attempt to attach to nullptr thread group` when working with materialized views. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3623) +* Fixed a crash when passing certain incorrect arguments to the `arrayReverse` function. [73e3a7b6](https://github.com/yandex/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) +* Fixed the buffer overflow in the `extractURLParameter` function. Improved performance. Added correct processing of strings containing zero bytes. [141e9799](https://github.com/yandex/ClickHouse/commit/141e9799e49201d84ea8e951d1bed4fb6d3dacb5) +* Fixed buffer overflow in the `lowerUTF8` and `upperUTF8` functions. Removed the ability to execute these functions over `FixedString` type arguments. [#3662](https://github.com/yandex/ClickHouse/pull/3662) +* Fixed a rare race condition when deleting `MergeTree` tables. [#3680](https://github.com/yandex/ClickHouse/pull/3680) +* Fixed a race condition when reading from `Buffer` tables and simultaneously performing `ALTER` or `DROP` on the target tables. [#3719](https://github.com/yandex/ClickHouse/pull/3719) +* Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [#3788](https://github.com/yandex/ClickHouse/pull/3788) + +### Improvements: + +* The server does not write the processed configuration files to the `/etc/clickhouse-server/` directory. Instead, it saves them in the `preprocessed_configs` directory inside `path`. This means that the `/etc/clickhouse-server/` directory doesn't have write access for the `clickhouse` user, which improves security. [#2443](https://github.com/yandex/ClickHouse/pull/2443) +* The `min_merge_bytes_to_use_direct_io` option is set to 10 GiB by default. A merge that forms large parts of tables from the MergeTree family will be performed in `O_DIRECT` mode, which prevents the cache resources from being cleared. [#3504](https://github.com/yandex/ClickHouse/pull/3504) +* Accelerated server start when there is a very large number of tables. [#3398](https://github.com/yandex/ClickHouse/pull/3398) +* Added a connection pool and HTTP `Keep-Alive` for connections between replicas. [#3594](https://github.com/yandex/ClickHouse/pull/3594) +* If the request syntax is invalid, the `400 Bad Request` code is returned in the `HTTP` interface (500 was returned previously). [31bc680a](https://github.com/yandex/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) +* The `join_default_strictness` option is set to `ALL` by default for compatibility. [120e2cbe](https://github.com/yandex/ClickHouse/commit/120e2cbe2ff4fbad626c28042d9b28781c805afe) +* Removed logging to `stderr` from the `re2` library for invalid or complex regular expressions. [#3723](https://github.com/yandex/ClickHouse/pull/3723) +* Added for the `Kafka` table engine: checks for subscriptions before beginning to read from Kafka; the kafka_max_block_size setting for the table. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3396) +* The `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, and `murmurHash3_64` functions now work for any number of arguments and for arguments in the form of tuples. [#3451](https://github.com/yandex/ClickHouse/pull/3451) [#3519](https://github.com/yandex/ClickHouse/pull/3519) +* The `arrayReverse` function now works with any types of arrays. [73e3a7b6](https://github.com/yandex/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) +* Added an optional parameter: the slot size for the `timeSlots` function. [Kirill Shvakov](https://github.com/yandex/ClickHouse/pull/3724) +* For `FULL` and `RIGHT JOIN`, the `max_block_size` setting is used for a stream of non-connected data from the right table. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3699) +* Added the `--secure` command line parameter in `clickhouse-benchmark` and `clickhouse-performance-test` to enable TLS. [#3688](https://github.com/yandex/ClickHouse/pull/3688) [#3690](https://github.com/yandex/ClickHouse/pull/3690) +* Type conversion when the structure of a `Buffer` type table does not match the structure of the destination table. [Vitaly Baranov](https://github.com/yandex/ClickHouse/pull/3603) +* Added the `tcp_keep_alive_timeout` option to enable keep-alive packets after inactivity for the specified time interval. [#3441](https://github.com/yandex/ClickHouse/pull/3441) +* Removed unnecessary quoting of values for the partition key in the `system.parts` table if it consists of a single column. [#3652](https://github.com/yandex/ClickHouse/pull/3652) +* The modulo function works for `Date` and `DateTime` data types. [#3385](https://github.com/yandex/ClickHouse/pull/3385) +* Added synonyms for the `POWER`, `LN`, `LCASE`, `UCASE`, `REPLACE`, `LOCATE`, `SUBSTR`, and `MID` functions. [#3774](https://github.com/yandex/ClickHouse/pull/3774) [#3763](https://github.com/yandex/ClickHouse/pull/3763) Some function names are case-insensitive for compatibility with the SQL standard. Added syntactic sugar `SUBSTRING(expr FROM start FOR length)` for compatibility with SQL. [#3804](https://github.com/yandex/ClickHouse/pull/3804) +* Added the ability to `mlock` memory pages corresponding to `clickhouse-server` executable code to prevent it from being forced out of memory. This feature is disabled by default. [#3553](https://github.com/yandex/ClickHouse/pull/3553) +* Improved performance when reading from `O_DIRECT` (with the `min_bytes_to_use_direct_io` option enabled). [#3405](https://github.com/yandex/ClickHouse/pull/3405) +* Improved performance of the `dictGet...OrDefault` function for a constant key argument and a non-constant default argument. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3563) +* The `firstSignificantSubdomain` function now processes the domains `gov`, `mil`, and `edu`. [Igor Hatarist](https://github.com/yandex/ClickHouse/pull/3601) Improved performance. [#3628](https://github.com/yandex/ClickHouse/pull/3628) +* Ability to specify custom environment variables for starting `clickhouse-server` using the `SYS-V init.d` script by defining `CLICKHOUSE_PROGRAM_ENV` in `/etc/default/clickhouse`. +[Pavlo Bashynskyi](https://github.com/yandex/ClickHouse/pull/3612) +* Correct return code for the clickhouse-server init script. [#3516](https://github.com/yandex/ClickHouse/pull/3516) +* The `system.metrics` table now has the `VersionInteger` metric, and `system.build_options` has the added line `VERSION_INTEGER`, which contains the numeric form of the ClickHouse version, such as `18016000`. [#3644](https://github.com/yandex/ClickHouse/pull/3644) +* Removed the ability to compare the `Date` type with a number to avoid potential errors like `date = 2018-12-17`, where quotes around the date are omitted by mistake. [#3687](https://github.com/yandex/ClickHouse/pull/3687) +* Fixed the behavior of stateful functions like `rowNumberInAllBlocks`. They previously output a result that was one number larger due to starting during query analysis. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3729) +* If the `force_restore_data` file can't be deleted, an error message is displayed. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3794) + +### Build improvements: + +* Updated the `jemalloc` library, which fixes a potential memory leak. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3557) +* Profiling with `jemalloc` is enabled by default in order to debug builds. [2cc82f5c](https://github.com/yandex/ClickHouse/commit/2cc82f5cbe266421cd4c1165286c2c47e5ffcb15) +* Added the ability to run integration tests when only `Docker` is installed on the system. [#3650](https://github.com/yandex/ClickHouse/pull/3650) +* Added the fuzz expression test in SELECT queries. [#3442](https://github.com/yandex/ClickHouse/pull/3442) +* Added a stress test for commits, which performs functional tests in parallel and in random order to detect more race conditions. [#3438](https://github.com/yandex/ClickHouse/pull/3438) +* Improved the method for starting clickhouse-server in a Docker image. [Elghazal Ahmed](https://github.com/yandex/ClickHouse/pull/3663) +* For a Docker image, added support for initializing databases using files in the `/docker-entrypoint-initdb.d` directory. [Konstantin Lebedev](https://github.com/yandex/ClickHouse/pull/3695) +* Fixes for builds on ARM. [#3709](https://github.com/yandex/ClickHouse/pull/3709) + +### Backward incompatible changes: + +* Removed the ability to compare the `Date` type with a number. Instead of `toDate('2018-12-18') = 17883`, you must use explicit type conversion `= toDate(17883)` [#3687](https://github.com/yandex/ClickHouse/pull/3687) + ## ClickHouse release 18.14.18, 2018-12-04 ### Bug fixes: From f6ce803eff25e67a56e0741ac134b38bf785cc57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 22:06:46 +0300 Subject: [PATCH 127/181] Better implementation of regexpQuoteMeta #3826 --- dbms/src/Functions/regexpQuoteMeta.cpp | 62 ++++++++++++-------------- 1 file changed, 29 insertions(+), 33 deletions(-) diff --git a/dbms/src/Functions/regexpQuoteMeta.cpp b/dbms/src/Functions/regexpQuoteMeta.cpp index 720e2029343..cc8f1791578 100644 --- a/dbms/src/Functions/regexpQuoteMeta.cpp +++ b/dbms/src/Functions/regexpQuoteMeta.cpp @@ -1,30 +1,17 @@ -#include -#include #include #include #include #include -#include -#include +#include -#include -#include - -#if USE_RE2_ST - #include // Y_IGNORE -#else - #define re2_st re2 -#endif namespace DB { -using namespace GatherUtils; namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } class FunctionRegexpQuoteMeta : public IFunction @@ -64,7 +51,7 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - const ColumnPtr column_string = block.getByPosition(arguments[0]).column; + const ColumnPtr & column_string = block.getByPosition(arguments[0]).column; const ColumnString * input = checkAndGetColumn(column_string.get()); if (!input) @@ -80,32 +67,41 @@ public: const ColumnString::Offsets & src_offsets = input->getOffsets(); - auto source = reinterpret_cast(input->getChars().data()); - auto dst = reinterpret_cast(dst_data.data()); - auto dst_pos = dst; + auto src_begin = reinterpret_cast(input->getChars().data()); + auto src_pos = src_begin; - size_t src_offset_prev = 0; - for (size_t row = 0; row < input_rows_count; ++row) + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { - size_t srclen = src_offsets[row] - src_offset_prev - 1; + /// NOTE This implementation slightly differs from re2::RE2::QuoteMeta. + /// It escapes zero byte as \0 instead of \x00 + /// and it escapes only required characters. + /// This is Ok. Look at comments in re2.cc - /// suboptimal, but uses original implementation from re2 - const auto & quoted = re2_st::RE2::QuoteMeta({source, srclen}); - const auto size = quoted.size() + 1; + const char * src_end = src_begin + src_offsets[row_idx] - 1; - size_t new_dst_size = dst_data.size() + size; - dst_data.resize(new_dst_size); - memcpy(dst_pos, quoted.c_str(), size); + while (true) + { + const char * next_src_pos = find_first_symbols<'\0', '\\', '|', '(', ')', '^', '$', '.', '[', '?', '*', '+', '{', ':', '-'>(src_pos, src_end); - source += srclen + 1; - dst_pos += size; + size_t bytes_to_copy = next_src_pos - src_pos; + size_t old_dst_size = dst_data.size(); + dst_data.resize(old_dst_size + bytes_to_copy); + memcpySmallAllowReadWriteOverflow15(dst_data.data() + old_dst_size, src_pos, bytes_to_copy); + src_pos = next_src_pos + 1; - dst_offsets[row] = new_dst_size; - src_offset_prev = src_offsets[row]; + if (next_src_pos == src_end) + { + dst_data.emplace_back('\0'); + break; + } + + dst_data.emplace_back('\\'); + dst_data.emplace_back(*next_src_pos); + } + + dst_offsets[row_idx] = dst_data.size(); } - dst_data.resize(dst_pos - dst); - block.getByPosition(result).column = std::move(dst_column); } From b6ebe2df190be050950aada96ef28480869d8405 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Dec 2018 22:13:12 +0300 Subject: [PATCH 128/181] Added missing test #3826 --- .../0_stateless/00807_regexp_quote_meta.reference | 12 ++++++++++++ .../queries/0_stateless/00807_regexp_quote_meta.sql | 12 ++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00807_regexp_quote_meta.reference create mode 100644 dbms/tests/queries/0_stateless/00807_regexp_quote_meta.sql diff --git a/dbms/tests/queries/0_stateless/00807_regexp_quote_meta.reference b/dbms/tests/queries/0_stateless/00807_regexp_quote_meta.reference new file mode 100644 index 00000000000..f58c91433e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00807_regexp_quote_meta.reference @@ -0,0 +1,12 @@ +hello +hel\\\\lo +h\\{ell}o +\\(h\\{ell}o\\) + +\\( +Hello\\( +\\(Hello +\\(\\(\\(\\(\\(\\(\\(\\(\\( +\\\\ +\\\0\\\\\\|\\(\\)\\^\\$\\.\\[\\?\\*\\+\\{ +1 diff --git a/dbms/tests/queries/0_stateless/00807_regexp_quote_meta.sql b/dbms/tests/queries/0_stateless/00807_regexp_quote_meta.sql new file mode 100644 index 00000000000..afac2d7bece --- /dev/null +++ b/dbms/tests/queries/0_stateless/00807_regexp_quote_meta.sql @@ -0,0 +1,12 @@ +SELECT regexpQuoteMeta('hello'); +SELECT regexpQuoteMeta('hel\\lo'); +SELECT regexpQuoteMeta('h{ell}o'); +SELECT regexpQuoteMeta('(h{ell}o)'); +SELECT regexpQuoteMeta(''); +SELECT regexpQuoteMeta('('); +SELECT regexpQuoteMeta('Hello('); +SELECT regexpQuoteMeta('(Hello'); +SELECT regexpQuoteMeta('((((((((('); +SELECT regexpQuoteMeta('\\'); +SELECT regexpQuoteMeta('\0\\|()^$.[?*+{'); +SELECT DISTINCT regexpQuoteMeta(toString(number)) = toString(number) FROM numbers(100000); From 1912bb8e7977546adf87a5a9fde7ed29d14e629e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 20 Dec 2018 22:34:29 +0300 Subject: [PATCH 129/181] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6058d40bab5..ad24a1a4592 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,7 +21,7 @@ ### Bug fixes: -* Fixes and performance improvements for the `LowCardinality` data type. `GROUP BY` using `LowCardinality(Nullable(...))`. Getting the values of `extremes`. Processing high-order functions. `LEFT JOIN ARRAY`. Distributed `GROUP BY`. Functions that return `Array`. Execution of `ORDER BY`. Writing to `Distributed` tables (nicelulu). Backward compatibility for `INSERT` queries from old clients that implement the `Native` protocol. Support for `LowCardinality` for `JOIN`. Improved performance when working in a single stream. [#3823](https://github.com/yandex/ClickHouse/pull/3823) [#3803](https://github.com/yandex/ClickHouse/pull/3803) [#3799](https://github.com/yandex/ClickHouse/pull/3799) [#3769](https://github.com/yandex/ClickHouse/pull/3769) [#3744](https://github.com/yandex/ClickHouse/pull/3744) [#3681](https://github.com/yandex/ClickHouse/pull/3681) [#3651](https://github.com/yandex/ClickHouse/pull/3651) [#3649](https://github.com/yandex/ClickHouse/pull/3649) [#3641](https://github.com/yandex/ClickHouse/pull/3641) [#3632](https://github.com/yandex/ClickHouse/pull/3632) [#3568](https://github.com/yandex/ClickHouse/pull/3568) [#3523](https://github.com/yandex/ClickHouse/pull/3523) [#3518](https://github.com/yandex/ClickHouse/pull/3518) +* Fixes and performance improvements for the `LowCardinality` data type. `GROUP BY` using `LowCardinality(Nullable(...))`. Getting the values of `extremes`. Processing high-order functions. `LEFT ARRAY JOIN`. Distributed `GROUP BY`. Functions that return `Array`. Execution of `ORDER BY`. Writing to `Distributed` tables (nicelulu). Backward compatibility for `INSERT` queries from old clients that implement the `Native` protocol. Support for `LowCardinality` for `JOIN`. Improved performance when working in a single stream. [#3823](https://github.com/yandex/ClickHouse/pull/3823) [#3803](https://github.com/yandex/ClickHouse/pull/3803) [#3799](https://github.com/yandex/ClickHouse/pull/3799) [#3769](https://github.com/yandex/ClickHouse/pull/3769) [#3744](https://github.com/yandex/ClickHouse/pull/3744) [#3681](https://github.com/yandex/ClickHouse/pull/3681) [#3651](https://github.com/yandex/ClickHouse/pull/3651) [#3649](https://github.com/yandex/ClickHouse/pull/3649) [#3641](https://github.com/yandex/ClickHouse/pull/3641) [#3632](https://github.com/yandex/ClickHouse/pull/3632) [#3568](https://github.com/yandex/ClickHouse/pull/3568) [#3523](https://github.com/yandex/ClickHouse/pull/3523) [#3518](https://github.com/yandex/ClickHouse/pull/3518) * Fixed how the `select_sequential_consistency` option works. Previously, when this setting was enabled, an incomplete result was sometimes returned after beginning to write to a new partition. [#2863](https://github.com/yandex/ClickHouse/pull/2863) * Databases are correctly specified when executing DDL `ON CLUSTER` queries and `ALTER UPDATE/DELETE`. [#3772](https://github.com/yandex/ClickHouse/pull/3772) [#3460](https://github.com/yandex/ClickHouse/pull/3460) * Databases are correctly specified for subqueries inside a VIEW. [#3521](https://github.com/yandex/ClickHouse/pull/3521) From 6c84afeb0f08166657770e12e06363a901863c61 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 20 Dec 2018 22:37:47 +0300 Subject: [PATCH 130/181] Update CHANGELOG.md --- CHANGELOG.md | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ad24a1a4592..9035fd7f1fb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,7 +9,7 @@ * Added functions for working with base64: `base64Encode`, `base64Decode`, `tryBase64Decode`. [Alexander Krasheninnikov](https://github.com/yandex/ClickHouse/pull/3350) * Now you can use a parameter to configure the precision of the `uniqCombined` aggregate function (select the number of HyperLogLog cells). [#3406](https://github.com/yandex/ClickHouse/pull/3406) * Added the `system.contributors` table that contains the names of everyone who made commits in ClickHouse. [#3452](https://github.com/yandex/ClickHouse/pull/3452) -* Added the ability to omit the partition for the `ALTER TABLE ... FREEZE` request in order to back up all partitions at once. [#3514](https://github.com/yandex/ClickHouse/pull/3514) +* Added the ability to omit the partition for the `ALTER TABLE ... FREEZE` query in order to back up all partitions at once. [#3514](https://github.com/yandex/ClickHouse/pull/3514) * Added `dictGet` and `dictGetOrDefault` functions that don't require specifying the type of return value. The type is determined automatically from the dictionary description. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3564) * Now you can specify comments for a column in the table description and change it using `ALTER`. [#3377](https://github.com/yandex/ClickHouse/pull/3377) * Reading is supported for `Join` type tables with simple keys. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3728) @@ -52,7 +52,7 @@ * The `min_merge_bytes_to_use_direct_io` option is set to 10 GiB by default. A merge that forms large parts of tables from the MergeTree family will be performed in `O_DIRECT` mode, which prevents the cache resources from being cleared. [#3504](https://github.com/yandex/ClickHouse/pull/3504) * Accelerated server start when there is a very large number of tables. [#3398](https://github.com/yandex/ClickHouse/pull/3398) * Added a connection pool and HTTP `Keep-Alive` for connections between replicas. [#3594](https://github.com/yandex/ClickHouse/pull/3594) -* If the request syntax is invalid, the `400 Bad Request` code is returned in the `HTTP` interface (500 was returned previously). [31bc680a](https://github.com/yandex/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) +* If the query syntax is invalid, the `400 Bad Request` code is returned in the `HTTP` interface (500 was returned previously). [31bc680a](https://github.com/yandex/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) * The `join_default_strictness` option is set to `ALL` by default for compatibility. [120e2cbe](https://github.com/yandex/ClickHouse/commit/120e2cbe2ff4fbad626c28042d9b28781c805afe) * Removed logging to `stderr` from the `re2` library for invalid or complex regular expressions. [#3723](https://github.com/yandex/ClickHouse/pull/3723) * Added for the `Kafka` table engine: checks for subscriptions before beginning to read from Kafka; the kafka_max_block_size setting for the table. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3396) @@ -185,7 +185,7 @@ ### Improvements: -* Significantly reduced memory consumption for requests with `ORDER BY` and `LIMIT`. See the `max_bytes_before_remerge_sort` setting. [#3205](https://github.com/yandex/ClickHouse/pull/3205) +* Significantly reduced memory consumption for queries with `ORDER BY` and `LIMIT`. See the `max_bytes_before_remerge_sort` setting. [#3205](https://github.com/yandex/ClickHouse/pull/3205) * In the absence of `JOIN` (`LEFT`, `INNER`, ...), `INNER JOIN` is assumed. [#3147](https://github.com/yandex/ClickHouse/pull/3147) * Qualified asterisks work correctly in queries with `JOIN`. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/3202) * The `ODBC` table engine correctly chooses the method for quoting identifiers in the SQL dialect of a remote database. [Alexandr Krasheninnikov](https://github.com/yandex/ClickHouse/pull/3210) @@ -222,7 +222,7 @@ * If after merging data parts, the checksum for the resulting part differs from the result of the same merge in another replica, the result of the merge is deleted and the data part is downloaded from the other replica (this is the correct behavior). But after downloading the data part, it couldn't be added to the working set because of an error that the part already exists (because the data part was deleted with some delay after the merge). This led to cyclical attempts to download the same data. [#3194](https://github.com/yandex/ClickHouse/pull/3194) * Fixed incorrect calculation of total memory consumption by queries (because of incorrect calculation, the `max_memory_usage_for_all_queries` setting worked incorrectly and the `MemoryTracking` metric had an incorrect value). This error occurred in version 18.12.13. [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3344) * Fixed the functionality of `CREATE TABLE ... ON CLUSTER ... AS SELECT ...` This error occurred in version 18.12.13. [#3247](https://github.com/yandex/ClickHouse/pull/3247) -* Fixed unnecessary preparation of data structures for `JOIN`s on the server that initiates the request if the `JOIN` is only performed on remote servers. [#3340](https://github.com/yandex/ClickHouse/pull/3340) +* Fixed unnecessary preparation of data structures for `JOIN`s on the server that initiates the query if the `JOIN` is only performed on remote servers. [#3340](https://github.com/yandex/ClickHouse/pull/3340) * Fixed bugs in the `Kafka` engine: deadlocks after exceptions when starting to read data, and locks upon completion [Marek Vavruša](https://github.com/yandex/ClickHouse/pull/3215). * For `Kafka` tables, the optional `schema` parameter was not passed (the schema of the `Cap'n'Proto` format). [Vojtech Splichal](https://github.com/yandex/ClickHouse/pull/3150) * If the ensemble of ZooKeeper servers has servers that accept the connection but then immediately close it instead of responding to the handshake, ClickHouse chooses to connect another server. Previously, this produced the error `Cannot read all data. Bytes read: 0. Bytes expected: 4.` and the server couldn't start. [8218cf3a](https://github.com/yandex/ClickHouse/commit/8218cf3a5f39a43401953769d6d12a0bb8d29da9) @@ -303,7 +303,7 @@ * Added the `DECIMAL(digits, scale)` data type (`Decimal32(scale)`, `Decimal64(scale)`, `Decimal128(scale)`). To enable it, use the setting `allow_experimental_decimal_type`. [#2846](https://github.com/yandex/ClickHouse/pull/2846) [#2970](https://github.com/yandex/ClickHouse/pull/2970) [#3008](https://github.com/yandex/ClickHouse/pull/3008) [#3047](https://github.com/yandex/ClickHouse/pull/3047) * New `WITH ROLLUP` modifier for `GROUP BY` (alternative syntax: `GROUP BY ROLLUP(...)`). [#2948](https://github.com/yandex/ClickHouse/pull/2948) -* In requests with JOIN, the star character expands to a list of columns in all tables, in compliance with the SQL standard. You can restore the old behavior by setting `asterisk_left_columns_only` to 1 on the user configuration level. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2787) +* In queries with JOIN, the star character expands to a list of columns in all tables, in compliance with the SQL standard. You can restore the old behavior by setting `asterisk_left_columns_only` to 1 on the user configuration level. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2787) * Added support for JOIN with table functions. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2907) * Autocomplete by pressing Tab in clickhouse-client. [Sergey Shcherbin](https://github.com/yandex/ClickHouse/pull/2447) * Ctrl+C in clickhouse-client clears a query that was entered. [#2877](https://github.com/yandex/ClickHouse/pull/2877) @@ -389,7 +389,7 @@ ### Backward incompatible changes: -* In requests with JOIN, the star character expands to a list of columns in all tables, in compliance with the SQL standard. You can restore the old behavior by setting `asterisk_left_columns_only` to 1 on the user configuration level. +* In queries with JOIN, the star character expands to a list of columns in all tables, in compliance with the SQL standard. You can restore the old behavior by setting `asterisk_left_columns_only` to 1 on the user configuration level. ### Build changes: @@ -433,7 +433,7 @@ * Fixed an error for concurrent `Set` or `Join`. [Amos Bird](https://github.com/yandex/ClickHouse/pull/2823) * Fixed the `Block structure mismatch in UNION stream: different number of columns` error that occurred for `UNION ALL` queries inside a sub-query if one of the `SELECT` queries contains duplicate column names. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2094) * Fixed a memory leak if an exception occurred when connecting to a MySQL server. -* Fixed incorrect clickhouse-client response code in case of a request error. +* Fixed incorrect clickhouse-client response code in case of a query error. * Fixed incorrect behavior of materialized views containing DISTINCT. [#2795](https://github.com/yandex/ClickHouse/issues/2795) ### Backward incompatible changes @@ -547,7 +547,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Fixed a problem with a very small timeout for sockets (one second) for reading and writing when sending and downloading replicated data, which made it impossible to download larger parts if there is a load on the network or disk (it resulted in cyclical attempts to download parts). This error occurred in version 1.1.54388. * Fixed issues when using chroot in ZooKeeper if you inserted duplicate data blocks in the table. * The `has` function now works correctly for an array with Nullable elements ([#2115](https://github.com/yandex/ClickHouse/issues/2115)). -* The `system.tables` table now works correctly when used in distributed queries. The `metadata_modification_time` and `engine_full` columns are now non-virtual. Fixed an error that occurred if only these columns were requested from the table. +* The `system.tables` table now works correctly when used in distributed queries. The `metadata_modification_time` and `engine_full` columns are now non-virtual. Fixed an error that occurred if only these columns were queried from the table. * Fixed how an empty `TinyLog` table works after inserting an empty data block ([#2563](https://github.com/yandex/ClickHouse/issues/2563)). * The `system.zookeeper` table works if the value of the node in ZooKeeper is NULL. @@ -796,7 +796,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Added the `parseDateTimeBestEffort`, `parseDateTimeBestEffortOrZero`, and `parseDateTimeBestEffortOrNull` functions to read the DateTime from a string containing text in a wide variety of possible formats. * Data can be partially reloaded from external dictionaries during updating (load just the records in which the value of the specified field greater than in the previous download) (Arsen Hakobyan). * Added the `cluster` table function. Example: `cluster(cluster_name, db, table)`. The `remote` table function can accept the cluster name as the first argument, if it is specified as an identifier. -* The `remote` and `cluster` table functions can be used in `INSERT` requests. +* The `remote` and `cluster` table functions can be used in `INSERT` queries. * Added the `create_table_query` and `engine_full` virtual columns to the `system.tables`table . The `metadata_modification_time` column is virtual. * Added the `data_path` and `metadata_path` columns to `system.tables`and` system.databases` tables, and added the `path` column to the `system.parts` and `system.parts_columns` tables. * Added additional information about merges in the `system.part_log` table. @@ -1135,7 +1135,7 @@ This release contains bug fixes for the previous release 1.1.54310: ### Please note when upgrading: -* There is now a higher default value for the MergeTree setting `max_bytes_to_merge_at_max_space_in_pool` (the maximum total size of data parts to merge, in bytes): it has increased from 100 GiB to 150 GiB. This might result in large merges running after the server upgrade, which could cause an increased load on the disk subsystem. If the free space available on the server is less than twice the total amount of the merges that are running, this will cause all other merges to stop running, including merges of small data parts. As a result, INSERT requests will fail with the message "Merges are processing significantly slower than inserts." Use the ` SELECT * FROM system.merges` request to monitor the situation. You can also check the `DiskSpaceReservedForMerge` metric in the `system.metrics` table, or in Graphite. You don't need to do anything to fix this, since the issue will resolve itself once the large merges finish. If you find this unacceptable, you can restore the previous value for the `max_bytes_to_merge_at_max_space_in_pool` setting. To do this, go to the section in config.xml, set ```107374182400` and restart the server. +* There is now a higher default value for the MergeTree setting `max_bytes_to_merge_at_max_space_in_pool` (the maximum total size of data parts to merge, in bytes): it has increased from 100 GiB to 150 GiB. This might result in large merges running after the server upgrade, which could cause an increased load on the disk subsystem. If the free space available on the server is less than twice the total amount of the merges that are running, this will cause all other merges to stop running, including merges of small data parts. As a result, INSERT queries will fail with the message "Merges are processing significantly slower than inserts." Use the ` SELECT * FROM system.merges` query to monitor the situation. You can also check the `DiskSpaceReservedForMerge` metric in the `system.metrics` table, or in Graphite. You don't need to do anything to fix this, since the issue will resolve itself once the large merges finish. If you find this unacceptable, you can restore the previous value for the `max_bytes_to_merge_at_max_space_in_pool` setting. To do this, go to the section in config.xml, set ```107374182400` and restart the server. ## ClickHouse release 1.1.54284, 2017-08-29 @@ -1228,7 +1228,7 @@ This release contains bug fixes for the previous release 1.1.54276: ### New features: * Distributed DDL (for example, `CREATE TABLE ON CLUSTER`) -* The replicated request `ALTER TABLE CLEAR COLUMN IN PARTITION.` +* The replicated query `ALTER TABLE CLEAR COLUMN IN PARTITION.` * The engine for Dictionary tables (access to dictionary data in the form of a table). * Dictionary database engine (this type of database automatically has Dictionary tables available for all the connected external dictionaries). * You can check for updates to the dictionary by sending a request to the source. From a5b84630dc270a9cdc307ad7005c34d0b662ea49 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 20 Dec 2018 22:52:04 +0300 Subject: [PATCH 131/181] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9035fd7f1fb..6d3b4a981b6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -49,7 +49,7 @@ ### Improvements: * The server does not write the processed configuration files to the `/etc/clickhouse-server/` directory. Instead, it saves them in the `preprocessed_configs` directory inside `path`. This means that the `/etc/clickhouse-server/` directory doesn't have write access for the `clickhouse` user, which improves security. [#2443](https://github.com/yandex/ClickHouse/pull/2443) -* The `min_merge_bytes_to_use_direct_io` option is set to 10 GiB by default. A merge that forms large parts of tables from the MergeTree family will be performed in `O_DIRECT` mode, which prevents the cache resources from being cleared. [#3504](https://github.com/yandex/ClickHouse/pull/3504) +* The `min_merge_bytes_to_use_direct_io` option is set to 10 GiB by default. A merge that forms large parts of tables from the MergeTree family will be performed in `O_DIRECT` mode, which prevents excessive page cache eviction. [#3504](https://github.com/yandex/ClickHouse/pull/3504) * Accelerated server start when there is a very large number of tables. [#3398](https://github.com/yandex/ClickHouse/pull/3398) * Added a connection pool and HTTP `Keep-Alive` for connections between replicas. [#3594](https://github.com/yandex/ClickHouse/pull/3594) * If the query syntax is invalid, the `400 Bad Request` code is returned in the `HTTP` interface (500 was returned previously). [31bc680a](https://github.com/yandex/ClickHouse/commit/31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab) From 414803e49caa8bf9adca3727c927dd96c909b7fc Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 20 Dec 2018 22:53:27 +0300 Subject: [PATCH 132/181] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6d3b4a981b6..784be3b4982 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -59,7 +59,7 @@ * The `cityHash64`, `farmHash64`, `metroHash64`, `sipHash64`, `halfMD5`, `murmurHash2_32`, `murmurHash2_64`, `murmurHash3_32`, and `murmurHash3_64` functions now work for any number of arguments and for arguments in the form of tuples. [#3451](https://github.com/yandex/ClickHouse/pull/3451) [#3519](https://github.com/yandex/ClickHouse/pull/3519) * The `arrayReverse` function now works with any types of arrays. [73e3a7b6](https://github.com/yandex/ClickHouse/commit/73e3a7b662161d6005e7727d8a711b930386b871) * Added an optional parameter: the slot size for the `timeSlots` function. [Kirill Shvakov](https://github.com/yandex/ClickHouse/pull/3724) -* For `FULL` and `RIGHT JOIN`, the `max_block_size` setting is used for a stream of non-connected data from the right table. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3699) +* For `FULL` and `RIGHT JOIN`, the `max_block_size` setting is used for a stream of non-joined data from the right table. [Amos Bird](https://github.com/yandex/ClickHouse/pull/3699) * Added the `--secure` command line parameter in `clickhouse-benchmark` and `clickhouse-performance-test` to enable TLS. [#3688](https://github.com/yandex/ClickHouse/pull/3688) [#3690](https://github.com/yandex/ClickHouse/pull/3690) * Type conversion when the structure of a `Buffer` type table does not match the structure of the destination table. [Vitaly Baranov](https://github.com/yandex/ClickHouse/pull/3603) * Added the `tcp_keep_alive_timeout` option to enable keep-alive packets after inactivity for the specified time interval. [#3441](https://github.com/yandex/ClickHouse/pull/3441) From d7ac2b3afd53fc73aa084e691b201e6faec14b41 Mon Sep 17 00:00:00 2001 From: ogorbacheva Date: Fri, 21 Dec 2018 13:21:24 +0300 Subject: [PATCH 133/181] Fix doc: edit ZooKeeper example in 'ru' and 'zh' (#3894) * changed the zookeper settings example * Fix doc: fix ZooKeeper example in ru and zh --- docs/ru/operations/server_settings/settings.md | 15 ++++++++++++++- docs/zh/operations/server_settings/settings.md | 15 ++++++++++++++- 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index b7e9095ebac..4996a283f8d 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -683,7 +683,20 @@ ClickHouse использует ZooKeeper для хранения метадан **Пример** ```xml - + + + example1 + 2181 + + + example2 + 2181 + + + example3 + 2181 + + ``` [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/server_settings/settings/) diff --git a/docs/zh/operations/server_settings/settings.md b/docs/zh/operations/server_settings/settings.md index 6e1e5e44e59..24a384a9087 100644 --- a/docs/zh/operations/server_settings/settings.md +++ b/docs/zh/operations/server_settings/settings.md @@ -680,7 +680,20 @@ For more information, see the section "[Replication](../../operations/table_engi **Example** ```xml - + + + example1 + 2181 + + + example2 + 2181 + + + example3 + 2181 + + ``` From 6ffdcb38a22ed8c1f9cdb59152c573f40d3f5cd2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 21 Dec 2018 19:37:33 +0800 Subject: [PATCH 134/181] Fix check data type with converts function --- dbms/src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index e461a4542f8..0e0bf218db3 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1157,7 +1157,7 @@ struct ToIntMonotonicity } /// If type is same, too. (Enum has separate case, because it is different data type) - if (checkAndGetDataType>(&type) || + if (checkAndGetDataType>(&type) || checkAndGetDataType>(&type)) return { true, true, true }; From 08d573d40217a104f43a446e8f84b32ba312a8c1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Dec 2018 17:23:39 +0300 Subject: [PATCH 135/181] Fixed error with unused variable #3826 --- dbms/src/Functions/trim.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index 10c9b6557aa..0484f369361 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -107,10 +107,10 @@ private: if constexpr (mode::trim_right) { - constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; const auto trim_right_size = size - chars_to_trim_left; #if __SSE4_2__ /// try to skip whitespace from right in blocks of up to 16 characters + constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; const auto trim_right_size_sse = trim_right_size - (trim_right_size % bytes_sse); while (mask == bytes_sse && chars_to_trim_right < trim_right_size_sse) { From d776d1164a09e95bfdb24050ffc1df93b45dbbd5 Mon Sep 17 00:00:00 2001 From: Boris Granveaud Date: Fri, 21 Dec 2018 15:53:00 +0100 Subject: [PATCH 136/181] support for IF EXISTS/IF NOT EXISTS in ALTER TABLE ADD/DROP/CLEAR/MODIFY/COMMENT COLUMN --- dbms/src/Parsers/ASTAlterQuery.cpp | 6 +- dbms/src/Parsers/ASTAlterQuery.h | 4 + dbms/src/Parsers/ParserAlterQuery.cpp | 17 ++++ dbms/src/Parsers/ParserAlterQuery.h | 10 +-- dbms/src/Storages/AlterCommands.cpp | 88 +++++++++++++------ dbms/src/Storages/AlterCommands.h | 15 +++- .../queries/0_stateless/00030_alter_table.sql | 7 ++ 7 files changed, 108 insertions(+), 39 deletions(-) diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index feec84d6e98..c5cdf1475e3 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -51,7 +51,7 @@ void ASTAlterCommand::formatImpl( if (type == ASTAlterCommand::ADD_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : ""); col_decl->formatImpl(settings, state, frame); /// AFTER @@ -64,7 +64,7 @@ void ASTAlterCommand::formatImpl( else if (type == ASTAlterCommand::DROP_COLUMN) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << (clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (settings.hilite ? hilite_none : ""); + << (clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); if (partition) { @@ -74,7 +74,7 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::MODIFY_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); col_decl->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::MODIFY_ORDER_BY) diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index e1cd74cb4b5..a6759482a56 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -78,6 +78,10 @@ public: bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata) + bool if_not_exists = false; /// option for ADD_COLUMN + + bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN + /** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition. */ String from; diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 83ad42ebbcb..b17467ed365 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -36,6 +36,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_partition("PARTITION"); ParserKeyword s_after("AFTER"); + ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_if_exists("IF EXISTS"); ParserKeyword s_from("FROM"); ParserKeyword s_in_partition("IN PARTITION"); ParserKeyword s_with("WITH"); @@ -57,6 +59,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected if (s_add_column.ignore(pos, expected)) { + if (s_if_not_exists.ignore(pos, expected)) + command->if_not_exists = true; + if (!parser_col_decl.parse(pos, command->col_decl, expected)) return false; @@ -77,6 +82,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_drop_column.ignore(pos, expected)) { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + if (!parser_name.parse(pos, command->column, expected)) return false; @@ -85,6 +93,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_clear_column.ignore(pos, expected)) { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + if (!parser_name.parse(pos, command->column, expected)) return false; @@ -190,6 +201,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_modify_column.ignore(pos, expected)) { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + if (!parser_modify_col_decl.parse(pos, command->col_decl, expected)) return false; @@ -224,6 +238,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_comment_column.ignore(pos, expected)) { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + if (!parser_name.parse(pos, command->column, expected)) return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 2eecfaf20d6..282a4277e17 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -8,12 +8,12 @@ namespace DB /** Query like this: * ALTER TABLE [db.]name [ON CLUSTER cluster] - * [ADD COLUMN col_name type [AFTER col_after],] - * [DROP COLUMN col_to_drop, ...] - * [CLEAR COLUMN col_to_clear [IN PARTITION partition],] - * [MODIFY COLUMN col_to_modify type, ...] + * [ADD COLUMN [IF NOT EXISTS] col_name type [AFTER col_after],] + * [DROP COLUMN [IF EXISTS] col_to_drop, ...] + * [CLEAR COLUMN [IF EXISTS] col_to_clear [IN PARTITION partition],] + * [MODIFY COLUMN [IF EXISTS] col_to_modify type, ...] * [MODIFY PRIMARY KEY (a, b, c...)] - * [COMMENT COLUMN col_name string] + * [COMMENT COLUMN [IF EXISTS] col_name string] * [DROP|DETACH|ATTACH PARTITION|PART partition, ...] * [FETCH PARTITION partition FROM ...] * [FREEZE [PARTITION] [WITH NAME name]] diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index ba5ed9e21bd..332ccfde3f0 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -52,6 +52,8 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ if (command_ast->column) command.after_column = typeid_cast(*command_ast->column).name; + command.if_not_exists = command_ast->if_not_exists; + return command; } else if (command_ast->type == ASTAlterCommand::DROP_COLUMN && !command_ast->partition) @@ -62,6 +64,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ AlterCommand command; command.type = AlterCommand::DROP_COLUMN; command.column_name = typeid_cast(*(command_ast->column)).name; + command.if_exists = command_ast->if_exists; return command; } else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN) @@ -88,6 +91,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ const auto & ast_comment = typeid_cast(*ast_col_decl.comment); command.comment = ast_comment.value.get(); } + command.if_exists = command_ast->if_exists; return command; } @@ -99,6 +103,7 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.column_name = ast_identifier.name; const auto & ast_comment = typeid_cast(*command_ast->comment); command.comment = ast_comment.value.get(); + command.if_exists = command_ast->if_exists; return command; } else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY) @@ -300,7 +305,8 @@ void AlterCommands::apply(ColumnsDescription & columns_description, ASTPtr & ord auto new_primary_key_ast = primary_key_ast; for (const AlterCommand & command : *this) - command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast); + if (!command.ignore) + command.apply(new_columns_description, new_order_by_ast, new_primary_key_ast); columns_description = std::move(new_columns_description); order_by_ast = std::move(new_order_by_ast); @@ -328,45 +334,61 @@ void AlterCommands::validate(const IStorage & table, const Context & context) if (command.type == AlterCommand::ADD_COLUMN) { if (std::end(all_columns) != column_it) - throw Exception{"Cannot add column " + column_name + ": column with this name already exists", ErrorCodes::ILLEGAL_COLUMN}; + { + if (command.if_not_exists) + command.ignore = true; + else + throw Exception{"Cannot add column " + column_name + ": column with this name already exists", ErrorCodes::ILLEGAL_COLUMN}; + } } else if (command.type == AlterCommand::MODIFY_COLUMN) { if (std::end(all_columns) == column_it) - throw Exception{"Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN}; + { + if (command.if_exists) + command.ignore = true; + else + throw Exception{"Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN}; + } - all_columns.erase(column_it); - defaults.erase(column_name); + if (!command.ignore) + { + all_columns.erase(column_it); + defaults.erase(column_name); + } } - /// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions - all_columns.emplace_back(column_name, command.data_type ? command.data_type : std::make_shared()); - - if (command.default_expression) + if (!command.ignore) { - if (command.data_type) + /// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions + all_columns.emplace_back(column_name, command.data_type ? command.data_type : std::make_shared()); + + if (command.default_expression) { - const auto & final_column_name = column_name; - const auto tmp_column_name = final_column_name + "_tmp"; - const auto column_type_raw_ptr = command.data_type.get(); + if (command.data_type) + { + const auto &final_column_name = column_name; + const auto tmp_column_name = final_column_name + "_tmp"; + const auto column_type_raw_ptr = command.data_type.get(); - default_expr_list->children.emplace_back(setAlias( - makeASTFunction("CAST", std::make_shared(tmp_column_name), - std::make_shared(column_type_raw_ptr->getName())), - final_column_name)); + default_expr_list->children.emplace_back(setAlias( + makeASTFunction("CAST", std::make_shared(tmp_column_name), + std::make_shared(column_type_raw_ptr->getName())), + final_column_name)); - default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name)); + default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name)); - defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command); - } - else - { - /// no type explicitly specified, will deduce later - default_expr_list->children.emplace_back( - setAlias(command.default_expression->clone(), column_name)); + defaulted_columns.emplace_back(NameAndTypePair{column_name, command.data_type}, &command); + } + else + { + /// no type explicitly specified, will deduce later + default_expr_list->children.emplace_back( + setAlias(command.default_expression->clone(), column_name)); - defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command); + defaulted_columns.emplace_back(NameAndTypePair{column_name, nullptr}, &command); + } } } } @@ -407,8 +429,13 @@ void AlterCommands::validate(const IStorage & table, const Context & context) } if (!found) - throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop", - ErrorCodes::ILLEGAL_COLUMN); + { + if (command.if_exists) + command.ignore = true; + else + throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop", + ErrorCodes::ILLEGAL_COLUMN); + } } else if (command.type == AlterCommand::COMMENT_COLUMN) { @@ -416,7 +443,10 @@ void AlterCommands::validate(const IStorage & table, const Context & context) std::bind(namesEqual, std::cref(command.column_name), std::placeholders::_1)); if (column_it == std::end(all_columns)) { - throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN}; + if (command.if_exists) + command.ignore = true; + else + throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN}; } } } diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index f1adbdaf9b0..f067d3811cc 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -43,15 +43,26 @@ struct AlterCommand /// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible. String after_column; + /// For DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN + bool if_exists; + + /// For ADD_COLUMN + bool if_not_exists; + /// For MODIFY_ORDER_BY ASTPtr order_by; + /// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist. + bool ignore = false; + AlterCommand() = default; AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, const ColumnDefaultKind default_kind, const ASTPtr & default_expression, - const String & after_column = String{}, const String & comment = "") // TODO: разобраться здесь с параметром по умолчанию + const String & after_column = String{}, const String & comment = "", + const bool if_exists = false, const bool if_not_exists = false) // TODO: разобраться здесь с параметром по умолчанию : type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind}, - default_expression{default_expression}, comment(comment), after_column{after_column} + default_expression{default_expression}, comment(comment), after_column{after_column}, + if_exists(if_exists), if_not_exists(if_not_exists) {} static std::optional parse(const ASTAlterCommand * command); diff --git a/dbms/tests/queries/0_stateless/00030_alter_table.sql b/dbms/tests/queries/0_stateless/00030_alter_table.sql index 231840818cf..cc5789b4040 100644 --- a/dbms/tests/queries/0_stateless/00030_alter_table.sql +++ b/dbms/tests/queries/0_stateless/00030_alter_table.sql @@ -23,6 +23,13 @@ ALTER TABLE test.alter_test DROP COLUMN NestedColumn.S; ALTER TABLE test.alter_test DROP COLUMN AddedNested1.B; +ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS Added0 UInt32; +ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1 Nested(A UInt32, B UInt64); +ALTER TABLE test.alter_test ADD COLUMN IF NOT EXISTS AddedNested1.C Array(String); +ALTER TABLE test.alter_test MODIFY COLUMN IF EXISTS ToDrop UInt64; +ALTER TABLE test.alter_test DROP COLUMN IF EXISTS ToDrop; +ALTER TABLE test.alter_test COMMENT COLUMN IF EXISTS ToDrop 'new comment'; + DESC TABLE test.alter_test; SELECT * FROM test.alter_test; From a8f09809f52385d23ea31f7820c3ac7bd3a976a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Dec 2018 19:00:07 +0300 Subject: [PATCH 137/181] Simplified logic with "IColumn::convertToFullColumnIfConst" (suggested by Amos Bird) [#CLICKHOUSE-2] --- dbms/src/Columns/ColumnArray.cpp | 13 ------- dbms/src/Columns/ColumnArray.h | 1 - dbms/src/Columns/ColumnNullable.cpp | 3 +- dbms/src/Columns/IColumn.h | 2 +- .../DataStreams/NativeBlockOutputStream.cpp | 7 +--- .../TotalsHavingBlockInputStream.cpp | 5 +-- dbms/src/DataStreams/materializeBlock.cpp | 4 +-- .../Functions/FunctionsExternalDictionaries.h | 20 +++-------- dbms/src/Functions/FunctionsFindCluster.h | 3 +- dbms/src/Functions/IFunction.cpp | 9 ++--- dbms/src/Functions/array.cpp | 3 +- dbms/src/Functions/arrayIndex.h | 12 ++----- dbms/src/Functions/arrayMap.cpp | 4 +-- dbms/src/Functions/if.cpp | 4 +-- dbms/src/Functions/materialize.cpp | 6 +--- dbms/src/Functions/tuple.cpp | 11 +++--- dbms/src/Interpreters/Aggregator.cpp | 18 +++------- dbms/src/Interpreters/ExpressionActions.cpp | 20 ++++------- dbms/src/Interpreters/Join.cpp | 27 +++------------ dbms/src/Interpreters/Set.cpp | 34 ++++--------------- .../Interpreters/evaluateMissingDefaults.cpp | 3 +- dbms/src/Storages/VirtualColumnUtils.cpp | 4 +-- 22 files changed, 46 insertions(+), 167 deletions(-) diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 18925ea165c..a3aa421d1c5 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -320,19 +320,6 @@ bool ColumnArray::hasEqualOffsets(const ColumnArray & other) const } -ColumnPtr ColumnArray::convertToFullColumnIfConst() const -{ - ColumnPtr new_data; - - if (ColumnPtr full_column = getData().convertToFullColumnIfConst()) - new_data = full_column; - else - new_data = data; - - return ColumnArray::create(new_data, offsets); -} - - void ColumnArray::getExtremes(Field & min, Field & max) const { min = Array(); diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index c2c17c17ed7..c73cc8faa1e 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -79,7 +79,6 @@ public: size_t byteSize() const override; size_t allocatedBytes() const override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; - ColumnPtr convertToFullColumnIfConst() const override; void getExtremes(Field & min, Field & max) const override; bool hasEqualOffsets(const ColumnArray & other) const; diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index 10a4519bf73..b88cf60581b 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -22,8 +22,7 @@ ColumnNullable::ColumnNullable(MutableColumnPtr && nested_column_, MutableColumn : nested_column(std::move(nested_column_)), null_map(std::move(null_map_)) { /// ColumnNullable cannot have constant nested column. But constant argument could be passed. Materialize it. - if (ColumnPtr nested_column_materialized = getNestedColumn().convertToFullColumnIfConst()) - nested_column = nested_column_materialized; + nested_column = getNestedColumn().convertToFullColumnIfConst(); if (!getNestedColumn().canBeInsideNullable()) throw Exception{getNestedColumn().getName() + " cannot be inside Nullable column", ErrorCodes::ILLEGAL_COLUMN}; diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 8f374ac526b..38df6ab3c38 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -45,7 +45,7 @@ public: /** If column isn't constant, returns nullptr (or itself). * If column is constant, transforms constant to full column (if column type allows such tranform) and return it. */ - virtual Ptr convertToFullColumnIfConst() const { return {}; } + virtual Ptr convertToFullColumnIfConst() const { return getPtr(); } /// If column isn't ColumnLowCardinality, return itself. /// If column is ColumnLowCardinality, transforms is to full column. diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index 1869badfe14..fb5aadb2fb3 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -46,12 +46,7 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) */ - ColumnPtr full_column; - - if (ColumnPtr converted = column->convertToFullColumnIfConst()) - full_column = converted; - else - full_column = column; + ColumnPtr full_column = column->convertToFullColumnIfConst(); IDataType::SerializeBinaryBulkSettings settings; settings.getter = [&ostr](IDataType::SubstreamPath) -> WriteBuffer * { return &ostr; }; diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp index 103d880f1d3..b7db949ca24 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp @@ -127,10 +127,7 @@ Block TotalsHavingBlockInputStream::readImpl() expression->execute(finalized); size_t filter_column_pos = finalized.getPositionByName(filter_column_name); - ColumnPtr filter_column_ptr = finalized.safeGetByPosition(filter_column_pos).column; - - if (ColumnPtr materialized = filter_column_ptr->convertToFullColumnIfConst()) - filter_column_ptr = materialized; + ColumnPtr filter_column_ptr = finalized.safeGetByPosition(filter_column_pos).column->convertToFullColumnIfConst(); FilterDescription filter_description(*filter_column_ptr); diff --git a/dbms/src/DataStreams/materializeBlock.cpp b/dbms/src/DataStreams/materializeBlock.cpp index a190baae9fa..60cd197912f 100644 --- a/dbms/src/DataStreams/materializeBlock.cpp +++ b/dbms/src/DataStreams/materializeBlock.cpp @@ -14,9 +14,7 @@ Block materializeBlock(const Block & block) for (size_t i = 0; i < columns; ++i) { auto & element = res.getByPosition(i); - auto & src = element.column; - if (ColumnPtr converted = src->convertToFullColumnIfConst()) - src = converted; + element.column = element.column->convertToFullColumnIfConst(); } return res; diff --git a/dbms/src/Functions/FunctionsExternalDictionaries.h b/dbms/src/Functions/FunctionsExternalDictionaries.h index a8116f44fbb..176cfe80eea 100644 --- a/dbms/src/Functions/FunctionsExternalDictionaries.h +++ b/dbms/src/Functions/FunctionsExternalDictionaries.h @@ -346,11 +346,8 @@ private: String attr_name = attr_name_col->getValue(); const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); - ColumnPtr key_col = key_col_with_type.column; - /// Functions in external dictionaries only support full-value (not constant) columns with keys. - if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst()) - key_col = key_col_materialized; + ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); if (checkColumn(key_col.get())) { @@ -578,11 +575,8 @@ private: String attr_name = attr_name_col->getValue(); const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); - ColumnPtr key_col = key_col_with_type.column; - /// Functions in external dictionaries only support full-value (not constant) columns with keys. - if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst()) - key_col = key_col_materialized; + ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); const auto & key_columns = typeid_cast(*key_col).getColumns(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); @@ -813,11 +807,9 @@ private: String attr_name = attr_name_col->getValue(); const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); - ColumnPtr key_col = key_col_with_type.column; /// Functions in external dictionaries only support full-value (not constant) columns with keys. - if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst()) - key_col = key_col_materialized; + ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); if (checkColumn(key_col.get())) { @@ -1079,11 +1071,9 @@ private: String attr_name = attr_name_col->getValue(); const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[2]); - ColumnPtr key_col = key_col_with_type.column; /// Functions in external dictionaries only support full-value (not constant) columns with keys. - if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst()) - key_col = key_col_materialized; + ColumnPtr key_col = key_col_with_type.column->convertToFullColumnIfConst(); const auto & key_columns = typeid_cast(*key_col).getColumns(); const auto & key_types = static_cast(*key_col_with_type.type).getElements(); @@ -1691,7 +1681,7 @@ static const PaddedPODArray & getColumnDataAsPaddedPODArray(const IColumn & c } } - const auto full_column = column.isColumnConst() ? column.convertToFullColumnIfConst() : column.getPtr(); + const auto full_column = column.convertToFullColumnIfConst(); // With type conversion and const columns we need to use backup storage here const auto size = full_column->size(); diff --git a/dbms/src/Functions/FunctionsFindCluster.h b/dbms/src/Functions/FunctionsFindCluster.h index 9e7e43c7dd9..5b20767c8aa 100644 --- a/dbms/src/Functions/FunctionsFindCluster.h +++ b/dbms/src/Functions/FunctionsFindCluster.h @@ -227,8 +227,7 @@ protected: bool executeOperationTyped(const IColumn * in_untyped, PaddedPODArray & dst, const IColumn * centroids_array_untyped) { const auto maybe_const = in_untyped->convertToFullColumnIfConst(); - if (maybe_const) - in_untyped = maybe_const.get(); + in_untyped = maybe_const.get(); const auto in_vector = checkAndGetColumn>(in_untyped); if (in_vector) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 08376a94f78..6b6186302f7 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -157,10 +157,7 @@ ColumnPtr wrapInNullable(const ColumnPtr & src, const Block & block, const Colum if (!result_null_map_column) return makeNullable(src); - if (src_not_nullable->isColumnConst()) - return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column); - else - return ColumnNullable::create(src_not_nullable, result_null_map_column); + return ColumnNullable::create(src_not_nullable->convertToFullColumnIfConst(), result_null_map_column); } @@ -431,9 +428,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows(), dry_run); - auto & keys = block_without_low_cardinality.safeGetByPosition(result).column; - if (auto full_column = keys->convertToFullColumnIfConst()) - keys = full_column; + auto keys = block_without_low_cardinality.safeGetByPosition(result).column->convertToFullColumnIfConst(); auto res_mut_dictionary = DataTypeLowCardinality::createColumnUnique(*res_low_cardinality_type->getDictionaryType()); ColumnPtr res_indexes = res_mut_dictionary->uniqueInsertRangeFrom(*keys, 0, keys->size()); diff --git a/dbms/src/Functions/array.cpp b/dbms/src/Functions/array.cpp index c0420b83db9..5205eff6b5f 100644 --- a/dbms/src/Functions/array.cpp +++ b/dbms/src/Functions/array.cpp @@ -69,8 +69,7 @@ public: if (!arg.type->equals(*elem_type)) preprocessed_column = castColumn(arg, elem_type, context); - if (ColumnPtr materialized_column = preprocessed_column->convertToFullColumnIfConst()) - preprocessed_column = materialized_column; + preprocessed_column = preprocessed_column->convertToFullColumnIfConst(); columns_holder[i] = std::move(preprocessed_column); columns[i] = columns_holder[i].get(); diff --git a/dbms/src/Functions/arrayIndex.h b/dbms/src/Functions/arrayIndex.h index 14429a1a4a5..fc8a8a08e47 100644 --- a/dbms/src/Functions/arrayIndex.h +++ b/dbms/src/Functions/arrayIndex.h @@ -838,15 +838,9 @@ private: null_map_data, nullptr); else { - /// If item_arg is tuple and have constants. - if (ColumnPtr materialized_tuple = item_arg.convertToFullColumnIfConst()) - ArrayIndexGenericImpl::vector( - col_nested, col_array->getOffsets(), *materialized_tuple, col_res->getData(), - null_map_data, null_map_item); - else - ArrayIndexGenericImpl::vector( - col_nested, col_array->getOffsets(), item_arg, col_res->getData(), - null_map_data, null_map_item); + ArrayIndexGenericImpl::vector( + col_nested, col_array->getOffsets(), *item_arg.convertToFullColumnIfConst(), col_res->getData(), + null_map_data, null_map_item); } block.getByPosition(result).column = std::move(col_res); diff --git a/dbms/src/Functions/arrayMap.cpp b/dbms/src/Functions/arrayMap.cpp index 534aaf51666..9575f520262 100644 --- a/dbms/src/Functions/arrayMap.cpp +++ b/dbms/src/Functions/arrayMap.cpp @@ -23,9 +23,7 @@ struct ArrayMapImpl static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped) { - return mapped->isColumnConst() - ? ColumnArray::create(mapped->convertToFullColumnIfConst(), array.getOffsetsPtr()) - : ColumnArray::create(mapped, array.getOffsetsPtr()); + return ColumnArray::create(mapped->convertToFullColumnIfConst(), array.getOffsetsPtr()); } }; diff --git a/dbms/src/Functions/if.cpp b/dbms/src/Functions/if.cpp index 42e8b65eb05..41a5277401c 100644 --- a/dbms/src/Functions/if.cpp +++ b/dbms/src/Functions/if.cpp @@ -638,9 +638,7 @@ private: static ColumnPtr materializeColumnIfConst(const ColumnPtr & column) { - if (ColumnPtr res = column->convertToFullColumnIfConst()) - return res; - return column; + return column->convertToFullColumnIfConst(); } static ColumnPtr makeNullableColumnIfNot(const ColumnPtr & column) diff --git a/dbms/src/Functions/materialize.cpp b/dbms/src/Functions/materialize.cpp index 4d3a1a57a77..552a2456f53 100644 --- a/dbms/src/Functions/materialize.cpp +++ b/dbms/src/Functions/materialize.cpp @@ -34,11 +34,7 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { - const auto & src = block.getByPosition(arguments[0]).column; - if (ColumnPtr converted = src->convertToFullColumnIfConst()) - block.getByPosition(result).column = converted; - else - block.getByPosition(result).column = src; + block.getByPosition(result).column = block.getByPosition(arguments[0]).column->convertToFullColumnIfConst(); } }; diff --git a/dbms/src/Functions/tuple.cpp b/dbms/src/Functions/tuple.cpp index 884ec880ffb..dcffff0ae9f 100644 --- a/dbms/src/Functions/tuple.cpp +++ b/dbms/src/Functions/tuple.cpp @@ -65,14 +65,11 @@ public: Columns tuple_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) { - tuple_columns[i] = block.getByPosition(arguments[i]).column; - /** If tuple is mixed of constant and not constant columns, - * convert all to non-constant columns, - * because many places in code expect all non-constant columns in non-constant tuple. - */ - if (ColumnPtr converted = tuple_columns[i]->convertToFullColumnIfConst()) - tuple_columns[i] = converted; + * convert all to non-constant columns, + * because many places in code expect all non-constant columns in non-constant tuple. + */ + tuple_columns[i] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst(); } block.getByPosition(result).column = ColumnTuple::create(tuple_columns); } diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 03f04d791a0..d90da905c2e 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -772,13 +772,8 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re /// Remember the columns we will work with for (size_t i = 0; i < params.keys_size; ++i) { - key_columns[i] = block.safeGetByPosition(params.keys[i]).column.get(); - - if (ColumnPtr converted = key_columns[i]->convertToFullColumnIfConst()) - { - materialized_columns.push_back(converted); - key_columns[i] = materialized_columns.back().get(); - } + materialized_columns.push_back(block.safeGetByPosition(params.keys[i]).column->convertToFullColumnIfConst()); + key_columns[i] = materialized_columns.back().get(); if (const auto * low_cardinality_column = typeid_cast(key_columns[i])) { @@ -797,13 +792,8 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re { for (size_t j = 0; j < aggregate_columns[i].size(); ++j) { - aggregate_columns[i][j] = block.safeGetByPosition(params.aggregates[i].arguments[j]).column.get(); - - if (ColumnPtr converted = aggregate_columns[i][j]->convertToFullColumnIfConst()) - { - materialized_columns.push_back(converted); - aggregate_columns[i][j] = materialized_columns.back().get(); - } + materialized_columns.push_back(block.safeGetByPosition(params.aggregates[i].arguments[j]).column->convertToFullColumnIfConst()); + aggregate_columns[i][j] = materialized_columns.back().get(); if (auto * col_low_cardinality = typeid_cast(aggregate_columns[i][j])) { diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 63f94591b05..0714c8a954a 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -375,10 +375,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const if (array_joined_columns.empty()) throw Exception("No arrays to join", ErrorCodes::LOGICAL_ERROR); - ColumnPtr any_array_ptr = block.getByName(*array_joined_columns.begin()).column; - if (ColumnPtr converted = any_array_ptr->convertToFullColumnIfConst()) - any_array_ptr = converted; - + ColumnPtr any_array_ptr = block.getByName(*array_joined_columns.begin()).column->convertToFullColumnIfConst(); const ColumnArray * any_array = typeid_cast(&*any_array_ptr); if (!any_array) throw Exception("ARRAY JOIN of not array: " + *array_joined_columns.begin(), ErrorCodes::TYPE_MISMATCH); @@ -416,10 +413,10 @@ void ExpressionAction::execute(Block & block, bool dry_run) const Block tmp_block{src_col, column_of_max_length, {{}, src_col.type, {}}}; function_arrayResize->build({src_col, column_of_max_length})->execute(tmp_block, {0, 1}, 2, rows); - any_array_ptr = src_col.column = tmp_block.safeGetByPosition(2).column; + src_col.column = tmp_block.safeGetByPosition(2).column; + any_array_ptr = src_col.column->convertToFullColumnIfConst(); } - if (ColumnPtr converted = any_array_ptr->convertToFullColumnIfConst()) - any_array_ptr = converted; + any_array = typeid_cast(&*any_array_ptr); } else if (array_join_is_left && !unaligned_array_join) @@ -434,10 +431,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const non_empty_array_columns[name] = tmp_block.safeGetByPosition(1).column; } - any_array_ptr = non_empty_array_columns.begin()->second; - if (ColumnPtr converted = any_array_ptr->convertToFullColumnIfConst()) - any_array_ptr = converted; - + any_array_ptr = non_empty_array_columns.begin()->second->convertToFullColumnIfConst(); any_array = &typeid_cast(*any_array_ptr); } @@ -452,9 +446,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const throw Exception("ARRAY JOIN of not array: " + current.name, ErrorCodes::TYPE_MISMATCH); ColumnPtr array_ptr = (array_join_is_left && !unaligned_array_join) ? non_empty_array_columns[current.name] : current.column; - - if (ColumnPtr converted = array_ptr->convertToFullColumnIfConst()) - array_ptr = converted; + array_ptr = array_ptr->convertToFullColumnIfConst(); const ColumnArray & array = typeid_cast(*array_ptr); if (!unaligned_array_join && !array.hasEqualOffsets(typeid_cast(*any_array_ptr))) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index d688a158562..9ab5f26f9e3 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -437,14 +437,8 @@ bool Join::insertFromBlock(const Block & block) /// Memoize key columns to work. for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_right[i]).column)); + materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_right[i]).column->convertToFullColumnIfConst())); key_columns[i] = materialized_columns.back().get(); - - if (ColumnPtr converted = key_columns[i]->convertToFullColumnIfConst()) - { - materialized_columns.emplace_back(converted); - key_columns[i] = materialized_columns.back().get(); - } } /// We will insert to the map only keys, where all components are not NULL. @@ -483,11 +477,7 @@ bool Join::insertFromBlock(const Block & block) /// Rare case, when joined columns are constant. To avoid code bloat, simply materialize them. for (size_t i = 0; i < size; ++i) - { - ColumnPtr col = stored_block->safeGetByPosition(i).column; - if (ColumnPtr converted = col->convertToFullColumnIfConst()) - stored_block->safeGetByPosition(i).column = converted; - } + stored_block->safeGetByPosition(i).column = stored_block->safeGetByPosition(i).column->convertToFullColumnIfConst(); /// In case of LEFT and FULL joins, if use_nulls, convert joined columns to Nullable. if (use_nulls && (kind == ASTTableJoin::Kind::Left || kind == ASTTableJoin::Kind::Full)) @@ -685,14 +675,8 @@ void Join::joinBlockImpl( /// Memoize key columns to work with. for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_left[i]).column)); + materialized_columns.emplace_back(recursiveRemoveLowCardinality(block.getByName(key_names_left[i]).column->convertToFullColumnIfConst())); key_columns[i] = materialized_columns.back().get(); - - if (ColumnPtr converted = key_columns[i]->convertToFullColumnIfConst()) - { - materialized_columns.emplace_back(converted); - key_columns[i] = materialized_columns.back().get(); - } } /// Keys with NULL value in any column won't join to anything. @@ -710,10 +694,7 @@ void Join::joinBlockImpl( { for (size_t i = 0; i < existing_columns; ++i) { - auto & col = block.getByPosition(i).column; - - if (ColumnPtr converted = col->convertToFullColumnIfConst()) - col = converted; + block.getByPosition(i).column = block.getByPosition(i).column->convertToFullColumnIfConst(); /// If use_nulls, convert left columns (except keys) to Nullable. if (use_nulls) diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 6bc99f3355d..022cd9bd404 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -121,15 +121,10 @@ void Set::setHeader(const Block & block) /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - key_columns.emplace_back(block.safeGetByPosition(i).column.get()); + materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst()); + key_columns.emplace_back(materialized_columns.back().get()); data_types.emplace_back(block.safeGetByPosition(i).type); - if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst()) - { - materialized_columns.emplace_back(converted); - key_columns.back() = materialized_columns.back().get(); - } - /// Convert low cardinality column to full. if (auto * low_cardinality_type = typeid_cast(data_types.back().get())) { @@ -175,20 +170,8 @@ bool Set::insertFromBlock(const Block & block) /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - key_columns.emplace_back(block.safeGetByPosition(i).column.get()); - - if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst()) - { - materialized_columns.emplace_back(converted); - key_columns.back() = materialized_columns.back().get(); - } - - /// Convert low cardinality column to full. - if (key_columns.back()->lowCardinality()) - { - materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality()); - key_columns.back() = materialized_columns.back().get(); - } + materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality()); + key_columns.emplace_back(materialized_columns.back().get()); } size_t rows = block.rows(); @@ -365,18 +348,13 @@ ColumnPtr Set::execute(const Block & block, bool negative) const for (size_t i = 0; i < num_key_columns; ++i) { - key_columns.push_back(block.safeGetByPosition(i).column.get()); - if (!removeNullable(data_types[i])->equals(*removeNullable(block.safeGetByPosition(i).type))) throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.safeGetByPosition(i).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH); - if (ColumnPtr converted = key_columns.back()->convertToFullColumnIfConst()) - { - materialized_columns.emplace_back(converted); - key_columns.back() = materialized_columns.back().get(); - } + materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst()); + key_columns.emplace_back() = materialized_columns.back().get(); } /// We will check existence in Set only for keys, where all components are not NULL. diff --git a/dbms/src/Interpreters/evaluateMissingDefaults.cpp b/dbms/src/Interpreters/evaluateMissingDefaults.cpp index 0b330fb00cc..33dce42ab8e 100644 --- a/dbms/src/Interpreters/evaluateMissingDefaults.cpp +++ b/dbms/src/Interpreters/evaluateMissingDefaults.cpp @@ -67,8 +67,7 @@ void evaluateMissingDefaults(Block & block, if (copy_block.has(col->name)) { auto evaluated_col = copy_block.getByName(col->name); - if (ColumnPtr converted = evaluated_col.column->convertToFullColumnIfConst()) - evaluated_col.column = converted; + evaluated_col.column = evaluated_col.column->convertToFullColumnIfConst(); block.insert(pos, std::move(evaluated_col)); } diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index ba7f7005d80..d78a7a36727 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -166,9 +166,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c /// Filter the block. String filter_column_name = expression_ast->getColumnName(); - ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column; - if (ColumnPtr converted = filter_column->convertToFullColumnIfConst()) - filter_column = converted; + ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst(); const IColumn::Filter & filter = typeid_cast(*filter_column).getData(); for (size_t i = 0; i < block.columns(); ++i) From 661a117b91f2a296d97d4fde93f451e45306323c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Dec 2018 19:03:40 +0300 Subject: [PATCH 138/181] Added assertion [#CLICKHOUSE-2] --- dbms/src/Storages/Kafka/StorageKafka.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 3d15259efcd..e125c694a6f 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -195,6 +195,9 @@ public: if (isCancelledOrThrowIfKilled() || !hasClaimed()) return {}; + if (!reader) + throw Exception("Logical error: reader is not initialized", ErrorCodes::LOGICAL_ERROR); + return reader->read(); } @@ -239,7 +242,7 @@ private: size_t max_block_size; Block sample_block; std::unique_ptr read_buf; - BlockInputStreamPtr reader = nullptr; + BlockInputStreamPtr reader; bool finalized = false; // Return true if consumer has been claimed by the stream From 2c6e49c3f36dca8a37013954fad9507881056ef6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Dec 2018 19:24:47 +0300 Subject: [PATCH 139/181] More checks [#CLICKHOUSE-2] --- dbms/src/Storages/Kafka/StorageKafka.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index e125c694a6f..fd129b38519 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -63,10 +63,10 @@ static const String CONFIG_PREFIX = "kafka"; class ReadBufferFromKafkaConsumer : public ReadBuffer { rd_kafka_t * consumer; - rd_kafka_message_t * current; - bool current_pending; + rd_kafka_message_t * current = nullptr; + bool current_pending = false; /// We've fetched "current" message and need to process it on the next iteration. Poco::Logger * log; - size_t read_messages; + size_t read_messages = 0; char row_delimiter; bool nextImpl() override @@ -97,6 +97,10 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer rd_kafka_message_destroy(msg); return nextImpl(); } + + if (msg->len && !msg->payload) + throw Exception("Logical error: nullptr message returned with non-zero length", ErrorCodes::LOGICAL_ERROR); + ++read_messages; // Now we've received a new message. Check if we need to produce a delimiter @@ -129,8 +133,7 @@ class ReadBufferFromKafkaConsumer : public ReadBuffer public: ReadBufferFromKafkaConsumer(rd_kafka_t * consumer_, Poco::Logger * log_, char row_delimiter_) - : ReadBuffer(nullptr, 0), consumer(consumer_), current(nullptr), - current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_) + : ReadBuffer(nullptr, 0), consumer(consumer_), log(log_), row_delimiter(row_delimiter_) { if (row_delimiter != '\0') LOG_TRACE(log, "Row delimiter is: " << row_delimiter); @@ -156,9 +159,8 @@ public: class KafkaBlockInputStream : public IProfilingBlockInputStream { public: - KafkaBlockInputStream(StorageKafka & storage_, const Context & context_, const String & schema, size_t max_block_size_) - : storage(storage_), consumer(nullptr), context(context_), max_block_size(max_block_size_) + : storage(storage_), context(context_), max_block_size(max_block_size_) { // Always skip unknown fields regardless of the context (JSON or TSKV) context.setSetting("input_format_skip_unknown_fields", 1u); From 0550e57bdfe8928096f7d3518f8c6c84e9b92aef Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 21 Dec 2018 19:41:12 +0300 Subject: [PATCH 140/181] add ru changelogs --- CHANGELOG_RU.md | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index a03d064cf4b..fa438686132 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,17 @@ +## ClickHouse release 18.16.1, 2018-12-21 + +### Исправления ошибок: + +* Исправлена проблема, приводившая к невозможности обновить словари с источником ODBC. [#3825](https://github.com/yandex/ClickHouse/issues/3825), [#3829](https://github.com/yandex/ClickHouse/issues/3829) +* JIT-компиляция агрегатных функций теперь работает с LowCardinality столбцами. [#3838](https://github.com/yandex/ClickHouse/issues/3838) + +### Улучшения: + +* Добавлена настройка `low_cardinality_allow_in_native_format` (по умолчанию включена). Если её выключить, столбцы LowCardinality в Native формате будут преобразовываться в соответствующий обычный тип при SELECT и из этого типа при INSERT. [#3879](https://github.com/yandex/ClickHouse/pull/3879) + +### Улучшения сборки: +* Исправления сборки под macOS и ARM. + ## ClickHouse release 18.16.0, 2018-12-14 ### Новые возможности: @@ -93,6 +107,16 @@ * Удалена возможность сравнения типа `Date` с числом, необходимо вместо `toDate('2018-12-18') = 17883`, использовать явное приведение типов `= toDate(17883)` [#3687](https://github.com/yandex/ClickHouse/pull/3687) +## ClickHouse release 18.14.19, 2018-12-19 + +### Исправления ошибок: + +* Исправлена проблема, приводившая к невозможности обновить словари с источником ODBC. [#3825](https://github.com/yandex/ClickHouse/issues/3825), [#3829](https://github.com/yandex/ClickHouse/issues/3829) +* Исправлен segfault в случае превышения ограничения `max_temporary_non_const_columns`. [#3788](https://github.com/yandex/ClickHouse/pull/3788) +* Корректное указание базы данных при выполнении DDL запросов `ON CLUSTER`. [#3460](https://github.com/yandex/ClickHouse/pull/3460) + +### Улучшения сборки: +* Исправления сборки под ARM. ## ClickHouse release 18.14.18, 2018-12-04 From 65a55a2c2d9f0ad83fe6605e07adaca93dbcf58e Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 21 Dec 2018 19:41:20 +0300 Subject: [PATCH 141/181] add en changelogs --- CHANGELOG.md | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 784be3b4982..87d478a62df 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,18 @@ +## ClickHouse release 18.16.1, 2018-12-21 + +### Bug fixes: + +* Fixed an error that led to problems with updating dictionaries with the ODBC source. [#3825](https://github.com/yandex/ClickHouse/issues/3825), [#3829](https://github.com/yandex/ClickHouse/issues/3829) +* JIT compilation of aggregate functions now works with LowCardinality columns. [#3838](https://github.com/yandex/ClickHouse/issues/3838) + +### Improvements: + +* Added the `low_cardinality_allow_in_native_format` setting (enabled by default). When disabled, LowCardinality columns will be converted to ordinary columns for SELECT queries and ordinary columns will be expected for INSERT queries. [#3879](https://github.com/yandex/ClickHouse/pull/3879) + +### Build improvements: + +* Fixes for builds on macOS and ARM. + ## ClickHouse release 18.16.0, 2018-12-14 ### New features: @@ -93,6 +108,18 @@ * Removed the ability to compare the `Date` type with a number. Instead of `toDate('2018-12-18') = 17883`, you must use explicit type conversion `= toDate(17883)` [#3687](https://github.com/yandex/ClickHouse/pull/3687) +## ClickHouse release 18.14.19, 2018-12-19 + +### Bug fixes: + +* Fixed an error that led to problems with updating dictionaries with the ODBC source. [#3825](https://github.com/yandex/ClickHouse/issues/3825), [#3829](https://github.com/yandex/ClickHouse/issues/3829) +* Databases are correctly specified when executing DDL `ON CLUSTER` queries. [#3460](https://github.com/yandex/ClickHouse/pull/3460) +* Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [#3788](https://github.com/yandex/ClickHouse/pull/3788) + +### Build improvements: + +* Fixes for builds on ARM. + ## ClickHouse release 18.14.18, 2018-12-04 ### Bug fixes: From 37065b6b8fbf0ab0f876828f0c5898968cecbae5 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 21 Dec 2018 20:28:21 +0300 Subject: [PATCH 142/181] prevent race condition when locking a distributed ALTER task for execution [#CLICKHOUSE-4221] --- dbms/src/Interpreters/DDLWorker.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 808ff9b1cf9..54fcffbea2a 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -691,17 +691,31 @@ void DDLWorker::processTaskAlter( auto lock = createSimpleZooKeeperLock(zookeeper, shard_path, "lock", task.host_id_str); pcg64 rng(randomSeed()); + auto is_already_executed = [&]() -> bool + { + String executed_by; + if (zookeeper->tryGet(is_executed_path, executed_by)) + { + is_executed_by_any_replica = true; + LOG_DEBUG(log, "Task " << task.entry_name << " has already been executed by another replica (" + << executed_by << ") of the same shard."); + return true; + } + + return false; + }; + static const size_t max_tries = 20; for (size_t num_tries = 0; num_tries < max_tries; ++num_tries) { - if (zookeeper->exists(is_executed_path)) - { - is_executed_by_any_replica = true; + if (is_already_executed()) break; - } if (lock->tryLock()) { + if (is_already_executed()) + break; + tryExecuteQuery(rewritten_query, task, task.execution_status); if (execute_on_leader_replica && task.execution_status.code == ErrorCodes::NOT_IMPLEMENTED) From 37570071185f7bd1f7cda9c6e8b06c00fb7d51a6 Mon Sep 17 00:00:00 2001 From: mf5137 Date: Fri, 21 Dec 2018 18:53:16 +0100 Subject: [PATCH 143/181] Adding xxHash64 and xxHash32 functions --- dbms/src/Functions/CMakeLists.txt | 5 +- dbms/src/Functions/FunctionsHashing.cpp | 2 + dbms/src/Functions/FunctionsHashing.h | 43 +++++++++++ .../0_stateless/00803_xxhash.reference | 54 +++++++++++++ .../queries/0_stateless/00803_xxhash.sql | 77 +++++++++++++++++++ 5 files changed, 179 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00803_xxhash.reference create mode 100644 dbms/tests/queries/0_stateless/00803_xxhash.sql diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 1307c47260e..b88996bd6f9 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -12,7 +12,7 @@ add_library(clickhouse_functions ${LINK_MODE} ${clickhouse_functions_sources}) target_link_libraries(clickhouse_functions PUBLIC - dbms + dbms PRIVATE clickhouse_dictionaries ${CONSISTENT_HASHING_LIBRARY} @@ -21,7 +21,8 @@ target_link_libraries(clickhouse_functions ${METROHASH_LIBRARIES} murmurhash ${BASE64_LIBRARY} - ${OPENSSL_CRYPTO_LIBRARY}) + ${OPENSSL_CRYPTO_LIBRARY} + ${LZ4_LIBRARY}) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR}) diff --git a/dbms/src/Functions/FunctionsHashing.cpp b/dbms/src/Functions/FunctionsHashing.cpp index bafd205e16c..708d6b71e68 100644 --- a/dbms/src/Functions/FunctionsHashing.cpp +++ b/dbms/src/Functions/FunctionsHashing.cpp @@ -25,5 +25,7 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 22c664d433e..f2319e1794f 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -116,6 +117,7 @@ struct HalfMD5Impl /// If true, it will use intHash32 or intHash64 to hash POD types. This behaviour is intended for better performance of some functions. /// Otherwise it will hash bytes in memory as a string using corresponding hash function. + static constexpr bool use_int_hash_for_pods = false; }; @@ -355,6 +357,44 @@ struct ImplMetroHash64 static constexpr bool use_int_hash_for_pods = true; }; +struct ImplXxHash32 +{ + static constexpr auto name = "xxHash32"; + using ReturnType = UInt32; + + static auto apply(const char * s, const size_t len) { return XXH32(s, len, 0); } + /** + * With current implementation with more than 1 arguments it will give the results + * non-reproducable from outside of CH. + * + * Proper way of combining several input is to use streaming mode of hash function + * https://github.com/Cyan4973/xxHash/issues/114#issuecomment-334908566 + * + * In common case doable by init_state / update_state / finalize_state + */ + static auto combineHashes(UInt32 h1, UInt32 h2) { return IntHash32Impl::apply(h1) ^ h2; } + + static constexpr bool use_int_hash_for_pods = false; +}; + + +struct ImplXxHash64 +{ + static constexpr auto name = "xxHash64"; + using ReturnType = UInt64; + using uint128_t = CityHash_v1_0_2::uint128; + + static auto apply(const char * s, const size_t len) { return XXH64(s, len, 0); } + + /* + With current implementation with more than 1 arguments it will give the results + non-reproducable from outside of CH. (see comment on ImplXxHash32). + */ + static auto combineHashes(UInt64 h1, UInt64 h2) { return CityHash_v1_0_2::Hash128to64(uint128_t(h1, h2)); } + + static constexpr bool use_int_hash_for_pods = false; +}; + template class FunctionStringHashFixedString : public IFunction @@ -978,4 +1018,7 @@ using FunctionMurmurHash2_64 = FunctionAnyHash; using FunctionMurmurHash3_32 = FunctionAnyHash; using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; +using FunctionXxHash32 = FunctionAnyHash; +using FunctionXxHash64 = FunctionAnyHash; + } diff --git a/dbms/tests/queries/0_stateless/00803_xxhash.reference b/dbms/tests/queries/0_stateless/00803_xxhash.reference new file mode 100644 index 00000000000..7432b657191 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00803_xxhash.reference @@ -0,0 +1,54 @@ +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 diff --git a/dbms/tests/queries/0_stateless/00803_xxhash.sql b/dbms/tests/queries/0_stateless/00803_xxhash.sql new file mode 100644 index 00000000000..2e6a1710e3c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00803_xxhash.sql @@ -0,0 +1,77 @@ +SELECT hex(xxHash64('')) = upper('ef46db3751d8e999'); +SELECT hex(xxHash32('')) = upper('02cc5d05'); + +SELECT hex(xxHash64('ABC')) = upper('e66ae7354fcfee98'); +SELECT hex(xxHash32('ABC')) = upper('80712ed5'); + +SELECT hex(xxHash64('xxhash')) = upper('32dd38952c4bc720'); + +-- + +SELECT xxHash64(NULL) is NULL; +SELECT xxHash64() = toUInt64(16324913028386710556); + +SELECT xxHash64(0) = toUInt64(16804241149081757544); +SELECT xxHash64(123456) = toUInt64(9049736899514479480); + +select xxHash64(toUInt8(0)) = xxHash64('\0'); +select xxHash64(toUInt16(0)) = xxHash64('\0\0'); +select xxHash64(toUInt32(0)) = xxHash64('\0\0\0\0'); +select xxHash64(toUInt64(0)) = xxHash64('\0\0\0\0\0\0\0\0'); + +SELECT xxHash64(CAST(3 AS UInt8)) = toUInt64(2244420788148980662); +SELECT xxHash64(CAST(1.2684 AS Float32)) = toUInt64(6662491266811474554); +SELECT xxHash64(CAST(-154477 AS Int64)) = toUInt64(1162348840373071858); + +SELECT xxHash64('') = toUInt64(17241709254077376921); +SELECT xxHash64('foo') = toUInt64(3728699739546630719); +SELECT xxHash64(CAST('foo' AS FixedString(3))) = xxHash64('foo'); +SELECT xxHash64(CAST('bar' AS FixedString(3))) = toUInt64(5234164152756840025); +SELECT xxHash64(x) = toUInt64(9962287286179718960) FROM (SELECT CAST(1 AS Enum8('a' = 1, 'b' = 2)) as x); + +SELECT xxHash64('\x01') = toUInt64(9962287286179718960); +SELECT xxHash64('\x02\0') = toUInt64(6482051057365497128); +SELECT xxHash64('\x03\0\0\0') = toUInt64(13361037350151369407); + +SELECT xxHash64(1) = toUInt64(9962287286179718960); +SELECT xxHash64(toUInt16(2)) = toUInt64(6482051057365497128); +SELECT xxHash64(toUInt32(3)) = toUInt64(13361037350151369407); + +SELECT xxHash64(1, 2, 3) = toUInt64(13728743482242651702); +SELECT xxHash64(1, 3, 2) = toUInt64(10226792638577471533); +SELECT xxHash64(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))) = toUInt64(3521288460171939489); + +-- + +SELECT xxHash32(NULL) is NULL; +SELECT xxHash32() = toUInt32(4263699484); + +SELECT xxHash32(0) = toUInt32(3479547966); +SELECT xxHash32(123456) = toUInt32(1434661961); + +select xxHash32(toUInt8(0)) = xxHash32('\0'); +select xxHash32(toUInt16(0)) = xxHash32('\0\0'); +select xxHash32(toUInt32(0)) = xxHash32('\0\0\0\0'); + +SELECT xxHash32(CAST(3 AS UInt8)) = toUInt32(565077562); +SELECT xxHash32(CAST(1.2684 AS Float32)) = toUInt32(3120514536); +SELECT xxHash32(CAST(-154477 AS Int32)) = toUInt32(3279223048); + +SELECT xxHash32('') = toUInt32(46947589); +SELECT xxHash32('foo') = toUInt32(3792637401); +SELECT xxHash32(CAST('foo' AS FixedString(3))) = xxHash32('foo'); +SELECT xxHash32(CAST('bar' AS FixedString(3))) = toUInt32(1101146924); +SELECT xxHash32(x) = toUInt32(949155633) FROM (SELECT CAST(1 AS Enum8('a' = 1, 'b' = 2)) as x); + +SELECT xxHash32('\x01') = toUInt32(949155633); +SELECT xxHash32('\x02\0') = toUInt32(332955956); +SELECT xxHash32('\x03\0\0\0') = toUInt32(2158931063); + +SELECT xxHash32(1) = toUInt32(949155633); +SELECT xxHash32(toUInt16(2)) = toUInt32(332955956); +SELECT xxHash32(toUInt32(3)) = toUInt32(2158931063); + +SELECT xxHash32(1, 2, 3) = toUInt32(441104368); +SELECT xxHash32(1, 3, 2) = toUInt32(912264289); +SELECT xxHash32(('a', [1, 2, 3], 4, (4, ['foo', 'bar'], 1, (1, 2)))) = toUInt32(1930126291); + From 06aa03f1a78fe216b987efdd0b74ab8e9b181b0e Mon Sep 17 00:00:00 2001 From: ogorbacheva Date: Fri, 21 Dec 2018 22:23:55 +0300 Subject: [PATCH 144/181] Doc fixes: remove all anchors (#3897) * Doc fixes: rm anchors * Doc fixes: rm anchors * Doc fixes: fix links * Doc fixes: fix the links --- docs/en/data_types/array.md | 4 +- docs/en/data_types/datetime.md | 4 +- docs/en/data_types/index.md | 4 +- .../aggregatefunction.md | 6 +- docs/en/data_types/nullable.md | 6 +- docs/en/data_types/tuple.md | 2 +- docs/en/interfaces/cli.md | 4 +- docs/en/interfaces/formats.md | 80 +- docs/en/interfaces/http.md | 4 +- docs/en/interfaces/index.md | 4 +- .../en/interfaces/third-party/integrations.md | 2 +- docs/en/operations/access_rights.md | 2 +- docs/en/operations/configuration_files.md | 4 +- docs/en/operations/quotas.md | 4 +- docs/en/operations/server_settings/index.md | 4 +- .../en/operations/server_settings/settings.md | 8 +- docs/en/operations/settings/index.md | 4 +- .../settings/permissions_for_queries.md | 14 +- .../operations/settings/query_complexity.md | 4 +- docs/en/operations/settings/settings.md | 20 +- .../table_engines/collapsingmergetree.md | 4 +- docs/en/operations/table_engines/file.md | 4 +- docs/en/operations/table_engines/index.md | 4 +- docs/en/operations/table_engines/kafka.md | 2 +- docs/en/operations/table_engines/mergetree.md | 8 +- .../operations/table_engines/replication.md | 6 +- .../table_engines/summingmergetree.md | 4 +- docs/en/operations/table_engines/url.md | 4 +- docs/en/operations/table_engines/view.md | 4 +- .../agg_functions/combinators.md | 8 +- docs/en/query_language/agg_functions/index.md | 4 +- .../agg_functions/parametric_functions.md | 6 +- .../query_language/agg_functions/reference.md | 10 +- docs/en/query_language/alter.md | 8 +- .../en/query_language/dicts/external_dicts.md | 4 +- .../dicts/external_dicts_dict.md | 4 +- .../dicts/external_dicts_dict_layout.md | 12 +- .../dicts/external_dicts_dict_sources.md | 857 +++++++++--------- .../en/query_language/dicts/internal_dicts.md | 4 +- .../en/query_language/functions/array_join.md | 4 +- .../functions/ext_dict_functions.md | 4 +- .../query_language/functions/in_functions.md | 2 +- docs/en/query_language/index.md | 2 +- docs/en/query_language/misc.md | 4 +- docs/en/query_language/operators.md | 10 +- docs/en/query_language/select.md | 14 +- docs/en/query_language/syntax.md | 4 +- .../en/query_language/table_functions/file.md | 2 +- docs/fa/data_types/index.md | 4 +- docs/fa/interfaces/cli.md | 4 +- docs/fa/interfaces/formats.md | 14 +- docs/fa/interfaces/index.md | 4 +- .../fa/interfaces/third-party/integrations.md | 2 +- docs/ru/data_types/array.md | 4 +- docs/ru/data_types/date.md | 4 +- docs/ru/data_types/datetime.md | 4 +- docs/ru/data_types/index.md | 4 +- .../aggregatefunction.md | 4 +- docs/ru/data_types/nullable.md | 4 +- docs/ru/interfaces/cli.md | 5 +- docs/ru/interfaces/formats.md | 79 +- docs/ru/interfaces/http.md | 4 +- docs/ru/interfaces/index.md | 4 +- .../ru/interfaces/third-party/integrations.md | 2 +- docs/ru/operations/access_rights.md | 2 +- docs/ru/operations/configuration_files.md | 4 +- docs/ru/operations/quotas.md | 4 +- docs/ru/operations/server_settings/index.md | 4 +- .../ru/operations/server_settings/settings.md | 8 +- docs/ru/operations/settings/index.md | 4 +- .../settings/permissions_for_queries.md | 12 +- .../operations/settings/query_complexity.md | 4 +- docs/ru/operations/settings/settings.md | 41 +- .../table_engines/collapsingmergetree.md | 4 +- docs/ru/operations/table_engines/file.md | 4 +- docs/ru/operations/table_engines/index.md | 4 +- docs/ru/operations/table_engines/kafka.md | 2 +- docs/ru/operations/table_engines/mergetree.md | 6 +- .../operations/table_engines/replication.md | 4 +- .../table_engines/summingmergetree.md | 2 +- docs/ru/operations/table_engines/url.md | 4 +- docs/ru/operations/utils/index.md | 2 +- .../agg_functions/combinators.md | 8 +- docs/ru/query_language/agg_functions/index.md | 4 +- .../agg_functions/parametric_functions.md | 4 +- .../query_language/agg_functions/reference.md | 16 +- docs/ru/query_language/alter.md | 8 +- .../ru/query_language/dicts/external_dicts.md | 4 +- .../dicts/external_dicts_dict.md | 4 +- .../dicts/external_dicts_dict_layout.md | 8 +- .../ru/query_language/dicts/internal_dicts.md | 4 +- .../ru/query_language/functions/array_join.md | 4 +- .../functions/ext_dict_functions.md | 4 +- .../functions/higher_order_functions.md | 4 +- .../query_language/functions/in_functions.md | 2 +- .../functions/ym_dict_functions.md | 4 +- docs/ru/query_language/index.md | 2 +- docs/ru/query_language/misc.md | 5 +- docs/ru/query_language/operators.md | 10 +- docs/ru/query_language/select.md | 12 +- docs/ru/query_language/syntax.md | 4 +- .../ru/query_language/table_functions/file.md | 2 +- docs/zh/data_types/array.md | 4 +- docs/zh/data_types/datetime.md | 4 +- docs/zh/data_types/index.md | 4 +- docs/zh/data_types/nullable.md | 6 +- docs/zh/data_types/tuple.md | 2 +- docs/zh/interfaces/cli.md | 4 +- docs/zh/interfaces/formats.md | 75 +- docs/zh/interfaces/index.md | 4 +- .../zh/interfaces/third-party/integrations.md | 2 +- docs/zh/operations/access_rights.md | 2 +- docs/zh/operations/configuration_files.md | 4 +- docs/zh/operations/quotas.md | 4 +- docs/zh/operations/server_settings/index.md | 4 +- .../zh/operations/server_settings/settings.md | 8 +- docs/zh/operations/settings/index.md | 4 +- .../operations/settings/query_complexity.md | 8 +- docs/zh/operations/settings/settings.md | 20 +- .../table_engines/collapsingmergetree.md | 4 +- docs/zh/operations/table_engines/file.md | 4 +- docs/zh/operations/table_engines/kafka.md | 2 +- docs/zh/operations/table_engines/mergetree.md | 6 +- .../operations/table_engines/replication.md | 4 +- .../table_engines/summingmergetree.md | 4 +- docs/zh/operations/table_engines/url.md | 4 +- docs/zh/query_language/create.md | 2 +- docs/zh/query_language/select.md | 14 +- 128 files changed, 706 insertions(+), 1058 deletions(-) diff --git a/docs/en/data_types/array.md b/docs/en/data_types/array.md index 18bce65bb03..5e2b2f55127 100644 --- a/docs/en/data_types/array.md +++ b/docs/en/data_types/array.md @@ -1,6 +1,4 @@ - - -# Array(T) +# Array(T) {#data_type-array} Array of `T`-type items. diff --git a/docs/en/data_types/datetime.md b/docs/en/data_types/datetime.md index c87eacbc644..ccaf44fa40d 100644 --- a/docs/en/data_types/datetime.md +++ b/docs/en/data_types/datetime.md @@ -1,6 +1,4 @@ - - -# DateTime +# DateTime {#data_type-datetime} Date with time. Stored in four bytes as a Unix timestamp (unsigned). Allows storing values in the same range as for the Date type. The minimal value is output as 0000-00-00 00:00:00. The time is stored with accuracy up to one second (without leap seconds). diff --git a/docs/en/data_types/index.md b/docs/en/data_types/index.md index 9acd30e11f2..b0f61096952 100644 --- a/docs/en/data_types/index.md +++ b/docs/en/data_types/index.md @@ -1,6 +1,4 @@ - - -# Data Types +# Data Types {#data_types} ClickHouse can store various types of data in table cells. diff --git a/docs/en/data_types/nested_data_structures/aggregatefunction.md b/docs/en/data_types/nested_data_structures/aggregatefunction.md index c1a0b7af3a8..4a02c5c7ad3 100644 --- a/docs/en/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/en/data_types/nested_data_structures/aggregatefunction.md @@ -1,6 +1,4 @@ - - -# AggregateFunction(name, types_of_arguments...) +# AggregateFunction(name, types_of_arguments...) {#data_type-aggregatefunction} The intermediate state of an aggregate function. To get it, use aggregate functions with the `-State` suffix. To get aggregated data in the future, you must use the same aggregate functions with the `-Merge`suffix. @@ -25,7 +23,7 @@ CREATE TABLE t ) ENGINE = ... ``` -[uniq](../../query_language/agg_functions/reference.md#agg_function-uniq), anyIf ([any](../../query_language/agg_functions/reference.md#agg_function-any)+[If](../../query_language/agg_functions/combinators.md#agg-functions-combine-if)) and [quantiles](../../query_language/agg_functions/reference.md) are the aggregate functions supported in ClickHouse. +[uniq](../../query_language/agg_functions/reference.md#agg_function-uniq), anyIf ([any](../../query_language/agg_functions/reference.md#agg_function-any)+[If](../../query_language/agg_functions/combinators.md#agg-functions-combinator-if)) and [quantiles](../../query_language/agg_functions/reference.md) are the aggregate functions supported in ClickHouse. ## Usage diff --git a/docs/en/data_types/nullable.md b/docs/en/data_types/nullable.md index a3e48720e86..1680f82e35d 100644 --- a/docs/en/data_types/nullable.md +++ b/docs/en/data_types/nullable.md @@ -1,10 +1,8 @@ - - -# Nullable(TypeName) +# Nullable(TypeName) {#data_type-nullable} Allows to store special marker ([NULL](../query_language/syntax.md)) that denotes "missing value" alongside normal values allowed by `TypeName`. For example, a `Nullable(Int8)` type column can store `Int8` type values, and the rows that don't have a value will store `NULL`. -For a `TypeName`, you can't use composite data types [Array](array.md#data_type is array) and [Tuple](tuple.md). Composite data types can contain `Nullable` type values, such as `Array(Nullable(Int8))`. +For a `TypeName`, you can't use composite data types [Array](array.md) and [Tuple](tuple.md). Composite data types can contain `Nullable` type values, such as `Array(Nullable(Int8))`. A `Nullable` type field can't be included in table indexes. diff --git a/docs/en/data_types/tuple.md b/docs/en/data_types/tuple.md index 789443aabed..ee6cd571ca8 100644 --- a/docs/en/data_types/tuple.md +++ b/docs/en/data_types/tuple.md @@ -3,7 +3,7 @@ A tuple of elements, each having an individual [type](index.md#data_types). -You can't store tuples in tables (other than Memory tables). They are used for temporary column grouping. Columns can be grouped when an IN expression is used in a query, and for specifying certain formal parameters of lambda functions. For more information, see the sections [IN operators](../query_language/select.md) and [Higher order functions](../query_language/functions/higher_order_functions.md#higher_order_functions). +You can't store tuples in tables (other than Memory tables). They are used for temporary column grouping. Columns can be grouped when an IN expression is used in a query, and for specifying certain formal parameters of lambda functions. For more information, see the sections [IN operators](../query_language/select.md) and [Higher order functions](../query_language/functions/higher_order_functions.md). Tuples can be the result of a query. In this case, for text formats other than JSON, values are comma-separated in brackets. In JSON formats, tuples are output as arrays (in square brackets). diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index c042a87ee00..caffafe61ca 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -65,9 +65,7 @@ You can cancel a long query by pressing Ctrl+C. However, you will still need to The command-line client allows passing external data (external temporary tables) for querying. For more information, see the section "External data for query processing". - - -## Configuring +## Configuring {#interfaces_cli_configuration} You can pass parameters to `clickhouse-client` (all parameters have a default value) using: diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index a21e8b53d2a..83a0a320f18 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -1,6 +1,4 @@ - - -# Formats for input and output data +# Formats for input and output data {#formats} ClickHouse can accept (`INSERT`) and return (`SELECT`) data in various formats. @@ -32,9 +30,7 @@ The table below lists supported formats and how they can be used in `INSERT` and | [XML](#xml) | ✗ | ✔ | | [CapnProto](#capnproto) | ✔ | ✔ | - - -## TabSeparated +## TabSeparated {#tabseparated} In TabSeparated format, data is written by row. Each row contains values separated by tabs. Each value is follow by a tab, except the last value in the row, which is followed by a line feed. Strictly Unix line feeds are assumed everywhere. The last row also must contain a line feed at the end. Values are written in text format, without enclosing quotation marks, and with special characters escaped. @@ -97,34 +93,29 @@ Arrays are written as a list of comma-separated values in square brackets. Numbe [NULL](../query_language/syntax.md) is formatted as `\N`. - - -## TabSeparatedRaw +## TabSeparatedRaw {#tabseparatedraw} Differs from `TabSeparated` format in that the rows are written without escaping. This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). This format is also available under the name `TSVRaw`. - -## TabSeparatedWithNames +## TabSeparatedWithNames {#tabseparatedwithnames} Differs from the `TabSeparated` format in that the column names are written in the first row. During parsing, the first row is completely ignored. You can't use column names to determine their position or to check their correctness. (Support for parsing the header row may be added in the future.) This format is also available under the name `TSVWithNames`. - -## TabSeparatedWithNamesAndTypes +## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. During parsing, the first and second rows are completely ignored. This format is also available under the name `TSVWithNamesAndTypes`. - -## TSKV +## TSKV {#tskv} Similar to TabSeparated, but outputs a value in name=value format. Names are escaped the same way as in TabSeparated format, and the = symbol is also escaped. @@ -157,19 +148,17 @@ Both data output and parsing are supported in this format. For parsing, any orde Parsing allows the presence of the additional field `tskv` without the equal sign or a value. This field is ignored. - - -## CSV +## CSV {#csv} Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)). -When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost). +When formatting, rows are enclosed in double quotes. A double quote inside a string is output as two double quotes in a row. There are no other rules for escaping characters. Date and date-time are enclosed in double quotes. Numbers are output without quotes. Values are separated by a delimiter character, which is `,` by default. The delimiter character is defined in the setting [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Rows are separated using the Unix line feed (LF). Arrays are serialized in CSV as follows: first the array is serialized to a string as in TabSeparated format, and then the resulting string is output to CSV in double quotes. Tuples in CSV format are serialized as separate columns (that is, their nesting in the tuple is lost). ``` clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -*By default, the delimiter is `,`. See the [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) setting for more information. +*By default, the delimiter is `,`. See the [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter) setting for more information. When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported. @@ -180,9 +169,8 @@ The CSV format supports the output of totals and extremes the same way as `TabSe ## CSVWithNames Also prints the header row, similar to `TabSeparatedWithNames`. - -## JSON +## JSON {#json} Outputs data in JSON format. Besides data tables, it also outputs column names and types, along with some additional information: the total number of output rows, and the number of rows that could have been output if there weren't a LIMIT. Example: @@ -271,9 +259,7 @@ ClickHouse supports [NULL](../query_language/syntax.md), which is displayed as ` See also the JSONEachRow format. - - -## JSONCompact +## JSONCompact {#jsoncompact} Differs from JSON only in that data rows are output in arrays, not in objects. @@ -318,9 +304,8 @@ Example: This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). See also the `JSONEachRow` format. - -## JSONEachRow +## JSONEachRow {#jsoneachrow} Outputs data as separate JSON objects for each row (newline delimited JSON). @@ -340,22 +325,19 @@ Outputs data as separate JSON objects for each row (newline delimited JSON). Unlike the JSON format, there is no substitution of invalid UTF-8 sequences. Any set of bytes can be output in the rows. This is necessary so that data can be formatted without losing any information. Values are escaped in the same way as for JSON. For parsing, any order is supported for the values of different columns. It is acceptable for some values to be omitted – they are treated as equal to their default values. In this case, zeros and blank rows are used as default values. Complex values that could be specified in the table are not supported as defaults. Whitespace between elements is ignored. If a comma is placed after the objects, it is ignored. Objects don't necessarily have to be separated by new lines. - -## Native +## Native {#native} The most efficient format. Data is written and read by blocks in binary format. For each block, the number of rows, number of columns, column names and types, and parts of columns in this block are recorded one after another. In other words, this format is "columnar" – it doesn't convert columns to rows. This is the format used in the native interface for interaction between servers, for using the command-line client, and for C++ clients. You can use this format to quickly generate dumps that can only be read by the ClickHouse DBMS. It doesn't make sense to work with this format yourself. - -## Null +## Null {#null} Nothing is output. However, the query is processed, and when using the command-line client, data is transmitted to the client. This is used for tests, including productivity testing. Obviously, this format is only appropriate for output, not for parsing. - -## Pretty +## Pretty {#pretty} Outputs data as Unicode-art tables, also using ANSI-escape sequences for setting colors in the terminal. A full grid of the table is drawn, and each row occupies two lines in the terminal. @@ -405,20 +387,16 @@ Extremes: └────────────┴─────────┘ ``` - - -## PrettyCompact +## PrettyCompact {#prettycompact} Differs from `Pretty` in that the grid is drawn between rows and the result is more compact. This format is used by default in the command-line client in interactive mode. - -## PrettyCompactMonoBlock +## PrettyCompactMonoBlock {#prettycompactmonoblock} Differs from [PrettyCompact](#prettycompact) in that up to 10,000 rows are buffered, then output as a single table, not by blocks. - -## PrettyNoEscapes +## PrettyNoEscapes {#prettynoescapes} Differs from Pretty in that ANSI-escape sequences aren't used. This is necessary for displaying this format in a browser, as well as for using the 'watch' command-line utility. @@ -437,14 +415,12 @@ The same as the previous setting. ### PrettySpaceNoEscapes The same as the previous setting. - -## PrettySpace +## PrettySpace {#prettyspace} Differs from [PrettyCompact](#prettycompact) in that whitespace (space characters) is used instead of the grid. - -## RowBinary +## RowBinary {#rowbinary} Formats and parses data by row in binary format. Rows and values are listed consecutively, without separators. This format is less efficient than the Native format, since it is row-based. @@ -467,9 +443,7 @@ The minimum set of characters that you need to escape when passing data in Value This is the format that is used in `INSERT INTO t VALUES ...`, but you can also use it for formatting query results. - - -## Vertical +## Vertical {#vertical} Prints each value on a separate line with the column name specified. This format is convenient for printing just one or a few rows, if each row consists of a large number of columns. @@ -490,9 +464,7 @@ y: ᴺᵁᴸᴸ This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). - - -## VerticalRaw +## VerticalRaw {#verticalraw} Differs from `Vertical` format in that the rows are not escaped. This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). @@ -521,9 +493,7 @@ Row 1: test: string with \'quotes\' and \t with some special \n characters ``` - - -## XML +## XML {#xml} XML format is suitable only for output, not for parsing. Example: @@ -596,9 +566,7 @@ In string values, the characters `<` and `&` are escaped as `<` and `&`. Arrays are output as `HelloWorld...`,and tuples as `HelloWorld...`. - - -## CapnProto +## CapnProto {#capnproto} Cap'n Proto is a binary message format similar to Protocol Buffers and Thrift, but not like JSON or MessagePack. diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 2d7b69f60fb..8dece39249f 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -1,6 +1,4 @@ - - -# HTTP Interface +# HTTP Interface {#http_interface} The HTTP interface lets you use ClickHouse on any platform from any programming language. We use it for working from Java and Perl, as well as shell scripts. In other departments, the HTTP interface is used from Perl, Python, and Go. The HTTP interface is more limited than the native interface, but it has better compatibility. diff --git a/docs/en/interfaces/index.md b/docs/en/interfaces/index.md index e11e1a16ebc..b18f8c85a97 100644 --- a/docs/en/interfaces/index.md +++ b/docs/en/interfaces/index.md @@ -1,6 +1,4 @@ - - -# Interfaces +# Interfaces {#interfaces} ClickHouse provides two network interfaces (both can be optionally wrapped in TLS for additional security): diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 0da0fcf14a8..daa2a73958f 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -49,7 +49,7 @@ - [RClickhouse](https://github.com/IMSMWU/RClickhouse) (uses [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) - Java - [Hadoop](http://hadoop.apache.org) - - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../jdbc.md)) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (uses [JDBC](../../query_language/table_functions/jdbc.md)) - Scala - [Akka](https://akka.io) - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) diff --git a/docs/en/operations/access_rights.md b/docs/en/operations/access_rights.md index 4764dead9fe..13458a39d47 100644 --- a/docs/en/operations/access_rights.md +++ b/docs/en/operations/access_rights.md @@ -96,7 +96,7 @@ Access to the `system` database is always allowed (since this database is used f The user can get a list of all databases and tables in them by using `SHOW` queries or system tables, even if access to individual databases isn't allowed. -Database access is not related to the [readonly](settings/query_complexity.md#readonly) setting. You can't grant full access to one database and `readonly` access to another one. +Database access is not related to the [readonly](settings/permissions_for_queries.md#settings_readonly) setting. You can't grant full access to one database and `readonly` access to another one. [Original article](https://clickhouse.yandex/docs/en/operations/access_rights/) diff --git a/docs/en/operations/configuration_files.md b/docs/en/operations/configuration_files.md index eb0e8c699a0..43a062e262f 100644 --- a/docs/en/operations/configuration_files.md +++ b/docs/en/operations/configuration_files.md @@ -1,6 +1,4 @@ - - -# Configuration Files +# Configuration Files {#configuration_files} The main server config file is `config.xml`. It resides in the `/etc/clickhouse-server/` directory. diff --git a/docs/en/operations/quotas.md b/docs/en/operations/quotas.md index e8cde130828..d5a48408d26 100644 --- a/docs/en/operations/quotas.md +++ b/docs/en/operations/quotas.md @@ -1,6 +1,4 @@ - - -# Quotas +# Quotas {#quotas} Quotas allow you to limit resource usage over a period of time, or simply track the use of resources. Quotas are set up in the user config. This is usually 'users.xml'. diff --git a/docs/en/operations/server_settings/index.md b/docs/en/operations/server_settings/index.md index 88f11c48f4b..3b721f95ef2 100644 --- a/docs/en/operations/server_settings/index.md +++ b/docs/en/operations/server_settings/index.md @@ -1,6 +1,4 @@ - - -# Server configuration parameters +# Server configuration parameters {#server_settings} This section contains descriptions of server settings that cannot be changed at the session or query level. diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index cc65063c70b..6a7ff2c0e52 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -120,7 +120,7 @@ The default is `true`. ## format_schema_path -The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#format_capnproto) format. +The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#capnproto) format. **Example** @@ -221,9 +221,7 @@ Opens `https://tabix.io/` when accessing ` http://localhost: http_port`. ``` - - -## include_from +## include_from {#server_settings-include_from} The path to the file with substitutions. @@ -641,7 +639,7 @@ The uncompressed cache is advantageous for very short queries in individual case 8589934592 ``` -## user_files_path +## user_files_path {#server_settings-user_files_path} The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 5676796fd10..73152183079 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -1,6 +1,4 @@ - - -# Settings +# Settings {#settings} There are multiple ways to make all the settings described below. Settings are configured in layers, so each subsequent layer redefines the previous settings. diff --git a/docs/en/operations/settings/permissions_for_queries.md b/docs/en/operations/settings/permissions_for_queries.md index 5a28bc67f50..6c4600f954d 100644 --- a/docs/en/operations/settings/permissions_for_queries.md +++ b/docs/en/operations/settings/permissions_for_queries.md @@ -1,6 +1,4 @@ - - -# Permissions for queries +# Permissions for queries {#permissions_for_queries} Queries in ClickHouse can be divided into several types: @@ -17,9 +15,7 @@ The following settings regulate user permissions by the type of query: `KILL QUERY` can be performed with any settings. - - -## readonly +## readonly {#settings_readonly} Restricts permissions for read data, write data and change settings queries. @@ -33,15 +29,13 @@ See how the queries are divided into types [above](#permissions_for_queries). After setting `readonly = 1`, the user can't change `readonly` and `allow_ddl` settings in the current session. -When using the `GET` method in the [HTTP interface](../../interfaces/http.md#http_interface), `readonly = 1` is set automatically. To modify data, use the `POST` method. +When using the `GET` method in the [HTTP interface](../../interfaces/http.md), `readonly = 1` is set automatically. To modify data, use the `POST` method. **Default value** 0 - - -## allow_ddl +## allow_ddl {#settings_allow_ddl} Allows/denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries. diff --git a/docs/en/operations/settings/query_complexity.md b/docs/en/operations/settings/query_complexity.md index 2b3bc459305..af982e243ec 100644 --- a/docs/en/operations/settings/query_complexity.md +++ b/docs/en/operations/settings/query_complexity.md @@ -16,9 +16,7 @@ It can take one of two values: `throw` or `break`. Restrictions on aggregation ( `any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. - - -## max_memory_usage +## max_memory_usage {#settings_max_memory_usage} The maximum amount of RAM to use for running a query on a single server. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3b4cf268579..59ba8189261 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -32,9 +32,7 @@ Used when performing `SELECT` from a distributed table that points to replicate By default, 1 (enabled). - - -## force_index_by_date +## force_index_by_date {#settings-force_index_by_date} Disables query execution if the index can't be used by date. @@ -123,9 +121,7 @@ Queries sent to ClickHouse with this setup are logged according to the rules in log_queries=1 - - -## max_insert_block_size +## max_insert_block_size {#settings-max_insert_block_size} The size of blocks to form for insertion into a table. This setting only applies in cases when the server forms the blocks. @@ -137,9 +133,7 @@ By default, it is 1,048,576. This is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allows sorting more data in RAM. - - -## max_replica_delay_for_distributed_queries +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} Disables lagging replicas for distributed queries. See "[Replication](../../operations/table_engines/replication.md)". @@ -149,7 +143,7 @@ Default value: 0 (off). Used when performing `SELECT` from a distributed table that points to replicated tables. -## max_threads {#max_threads} +## max_threads {#settings-max_threads} The maximum number of query processing threads @@ -273,7 +267,7 @@ This parameter is useful when you are using formats that require a schema defini ## stream_flush_interval_ms -Works for tables with streaming in the case of a timeout, or when a thread generates[max_insert_block_size](#max-insert-block-size) rows. +Works for tables with streaming in the case of a timeout, or when a thread generates[max_insert_block_size](#settings-max_insert_block_size) rows. The default value is 7500. @@ -352,9 +346,7 @@ It works for JSONEachRow and TSKV formats. If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. - - -## format_csv_delimiter +## format_csv_delimiter {#settings-format_csv_delimiter} The character interpreted as a delimiter in the CSV data. By default, the delimiter is `,`. diff --git a/docs/en/operations/table_engines/collapsingmergetree.md b/docs/en/operations/table_engines/collapsingmergetree.md index 1c397165014..b6aa9b4ce2c 100644 --- a/docs/en/operations/table_engines/collapsingmergetree.md +++ b/docs/en/operations/table_engines/collapsingmergetree.md @@ -1,6 +1,4 @@ - - -# CollapsingMergeTree +# CollapsingMergeTree {#table_engine-collapsingmergetree} The engine inherits from [MergeTree](mergetree.md) and adds the logic of rows collapsing to data parts merge algorithm. diff --git a/docs/en/operations/table_engines/file.md b/docs/en/operations/table_engines/file.md index a394db256db..c63d78a2ba3 100644 --- a/docs/en/operations/table_engines/file.md +++ b/docs/en/operations/table_engines/file.md @@ -1,6 +1,4 @@ - - -# File(InputFormat) +# File(InputFormat) {#table_engines-file} The data source is a file that stores data in one of the supported input formats (TabSeparated, Native, etc.). diff --git a/docs/en/operations/table_engines/index.md b/docs/en/operations/table_engines/index.md index 21909e3ce26..e370e6173bb 100644 --- a/docs/en/operations/table_engines/index.md +++ b/docs/en/operations/table_engines/index.md @@ -1,6 +1,4 @@ - - -# Table engines +# Table engines {#table_engines} The table engine (type of table) determines: diff --git a/docs/en/operations/table_engines/kafka.md b/docs/en/operations/table_engines/kafka.md index 9a0756d1ea7..d668def503e 100644 --- a/docs/en/operations/table_engines/kafka.md +++ b/docs/en/operations/table_engines/kafka.md @@ -106,7 +106,7 @@ Example: SELECT level, sum(total) FROM daily GROUP BY level; ``` -To improve performance, received messages are grouped into blocks the size of [max_insert_block_size](../settings/settings.md#settings-settings-max_insert_block_size). If the block wasn't formed within [stream_flush_interval_ms](../settings/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block. +To improve performance, received messages are grouped into blocks the size of [max_insert_block_size](../settings/settings.md#settings-max_insert_block_size). If the block wasn't formed within [stream_flush_interval_ms](../settings/settings.md) milliseconds, the data will be flushed to the table regardless of the completeness of the block. To stop receiving topic data or to change the conversion logic, detach the materialized view: diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 0aba93e4591..14e5fd19fe2 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -1,6 +1,4 @@ - - -# MergeTree +# MergeTree {#table_engines-mergetree} The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most robust ClickHousе table engines. @@ -79,7 +77,7 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa In the example, we set partitioning by month. -We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If, when selecting the data, you define a [SAMPLE](../../query_language/select.md#sample) clause, ClickHouse will return an evenly pseudorandom data sample for a subset of users. +We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If, when selecting the data, you define a [SAMPLE](../../query_language/select.md#select-sample-clause) clause, ClickHouse will return an evenly pseudorandom data sample for a subset of users. `index_granularity` could be omitted because 8192 is the default value. @@ -223,7 +221,7 @@ In the example below, the index can't be used. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../settings/settings.md#settings-settings-force_index_by_date) and [force_primary_key](../settings/settings.md). +To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../settings/settings.md#settings-force_index_by_date) and [force_primary_key](../settings/settings.md). The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. diff --git a/docs/en/operations/table_engines/replication.md b/docs/en/operations/table_engines/replication.md index ef4d8a976ab..7a0c35acbd2 100644 --- a/docs/en/operations/table_engines/replication.md +++ b/docs/en/operations/table_engines/replication.md @@ -1,6 +1,4 @@ - - -# Data Replication +# Data Replication {#table_engines-replication} Replication is only supported for tables in the MergeTree family: @@ -49,7 +47,7 @@ You can specify any existing ZooKeeper cluster and the system will use a directo If ZooKeeper isn't set in the config file, you can't create replicated tables, and any existing replicated tables will be read-only. -ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../settings/settings.md#settings_settings_max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md). +ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md). For each `INSERT` query, approximately ten entries are added to ZooKeeper through several transactions. (To be more precise, this is for each inserted block of data; an INSERT query contains one block or one block per `max_insert_block_size = 1048576` rows.) This leads to slightly longer latencies for `INSERT` compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one `INSERT` per second, it doesn't create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred `INSERTs` per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. diff --git a/docs/en/operations/table_engines/summingmergetree.md b/docs/en/operations/table_engines/summingmergetree.md index 286507d8b9c..efea8da74be 100644 --- a/docs/en/operations/table_engines/summingmergetree.md +++ b/docs/en/operations/table_engines/summingmergetree.md @@ -90,7 +90,7 @@ SELECT key, sum(value) FROM summtt GROUP BY key When data are inserted into a table, they are saved as-is. Clickhouse merges the inserted parts of data periodically and this is when rows with the same primary key are summed and replaced with one for each resulting part of data. -ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) an aggregate function [sum()](../../query_language/agg_functions/reference.md#sum) and `GROUP BY` clause should be used in a query as described in the example above. +ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) an aggregate function [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) and `GROUP BY` clause should be used in a query as described in the example above. ### Common rules for summation @@ -104,7 +104,7 @@ The values are not summarized for columns in the primary key. ### The Summation in the AggregateFunction Columns -For columns of [AggregateFunction type](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction) ClickHouse behaves as [AggregatingMergeTree](aggregatingmergetree.md) engine aggregating according to the function. +For columns of [AggregateFunction type](../../data_types/nested_data_structures/aggregatefunction.md) ClickHouse behaves as [AggregatingMergeTree](aggregatingmergetree.md) engine aggregating according to the function. ### Nested Structures diff --git a/docs/en/operations/table_engines/url.md b/docs/en/operations/table_engines/url.md index 77e690c4de5..1cf803c763c 100644 --- a/docs/en/operations/table_engines/url.md +++ b/docs/en/operations/table_engines/url.md @@ -1,6 +1,4 @@ - - -# URL(URL, Format) +# URL(URL, Format) {#table_engines-url} Manages data on a remote HTTP/HTTPS server. This engine is similar to the [File](file.md) engine. diff --git a/docs/en/operations/table_engines/view.md b/docs/en/operations/table_engines/view.md index 204c0d9a8de..42e46148ce5 100644 --- a/docs/en/operations/table_engines/view.md +++ b/docs/en/operations/table_engines/view.md @@ -1,6 +1,4 @@ - - -# View +# View {#table_engines-view} Used for implementing views (for more information, see the `CREATE VIEW query`). It does not store data, but only stores the specified `SELECT` query. When reading from a table, it runs this query (and deletes all unnecessary columns from the query). diff --git a/docs/en/query_language/agg_functions/combinators.md b/docs/en/query_language/agg_functions/combinators.md index b63beee4a27..bbd49a5c6f7 100644 --- a/docs/en/query_language/agg_functions/combinators.md +++ b/docs/en/query_language/agg_functions/combinators.md @@ -1,12 +1,8 @@ - - -# Aggregate function combinators +# Aggregate function combinators {#aggregate_functions_combinators} The name of an aggregate function can have a suffix appended to it. This changes the way the aggregate function works. - - -## -If +## -If {#agg-functions-combinator-if} The suffix -If can be appended to the name of any aggregate function. In this case, the aggregate function accepts an extra argument – a condition (Uint8 type). The aggregate function processes only the rows that trigger the condition. If the condition was not triggered even once, it returns a default value (usually zeros or empty strings). diff --git a/docs/en/query_language/agg_functions/index.md b/docs/en/query_language/agg_functions/index.md index a3cb2132ed8..93da97357d5 100644 --- a/docs/en/query_language/agg_functions/index.md +++ b/docs/en/query_language/agg_functions/index.md @@ -1,6 +1,4 @@ - - -# Aggregate functions +# Aggregate functions {#aggregate_functions} Aggregate functions work in the [normal](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) way as expected by database experts. diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index 61951379d56..5a49b077089 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -1,6 +1,4 @@ - - -# Parametric aggregate functions +# Parametric aggregate functions {#aggregate_functions_parametric} Some aggregate functions can accept not only argument columns (used for compression), but a set of parameters – constants for initialization. The syntax is two pairs of brackets instead of one. The first is for parameters, and the second is for arguments. @@ -61,7 +59,7 @@ windowFunnel(window)(timestamp, cond1, cond2, cond3, ...) **Parameters:** - `window` — Length of the sliding window in seconds. -- `timestamp` — Name of the column containing the timestamp. Data type: [DateTime](../../data_types/datetime.md#data_type-datetime) or [UInt32](../../data_types/int_uint.md). +- `timestamp` — Name of the column containing the timestamp. Data type: [DateTime](../../data_types/datetime.md) or [UInt32](../../data_types/int_uint.md). - `cond1`, `cond2`... — Conditions or data describing the chain of events. Data type: `UInt8`. Values can be 0 or 1. **Algorithm** diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 6b214e71acc..882371dedc0 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -8,9 +8,7 @@ The syntax `COUNT(DISTINCT x)` is not supported. The separate `uniq` aggregate f A `SELECT count() FROM table` query is not optimized, because the number of entries in the table is not stored separately. It will select some small column from the table and count the number of values in it. - - -## any(x) +## any(x) {#agg_function-any} Selects the first encountered value. The query can be executed in any order and even in a different order each time, so the result of this function is indeterminate. @@ -213,7 +211,7 @@ SELECT argMin(user, salary) FROM salary Calculates the 'arg' value for a maximum 'val' value. If there are several different values of 'arg' for maximum values of 'val', the first of these values encountered is output. -## sum(x) +## sum(x) {#agg_function-sum} Calculates the sum. Only works for numbers. @@ -267,9 +265,7 @@ Calculates the average. Only works for numbers. The result is always Float64. - - -## uniq(x) +## uniq(x) {#agg_function-uniq} Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments. diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index 9457fafc71d..4d146f55cbf 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -1,6 +1,4 @@ - - -## ALTER +## ALTER {#query_language_queries_alter} The `ALTER` query is only supported for `*MergeTree` tables, as well as `Merge`and`Distributed`. The query has several variations. @@ -230,9 +228,7 @@ For non-replicatable tables, all `ALTER` queries are performed synchronously. Fo For `ALTER ... ATTACH|DETACH|DROP` queries, you can use the `replication_alter_partitions_sync` setting to set up waiting. Possible values: `0` – do not wait; `1` – only wait for own execution (default); `2` – wait for all. - - -### Mutations +### Mutations {#query_language_queries_show_databases} Mutations are an ALTER query variant that allows changing or deleting rows in a table. In contrast to standard `UPDATE` and `DELETE` queries that are intended for point data changes, mutations are intended for heavy operations that change a lot of rows in a table. diff --git a/docs/en/query_language/dicts/external_dicts.md b/docs/en/query_language/dicts/external_dicts.md index 2c98ff18187..9af87dcc516 100644 --- a/docs/en/query_language/dicts/external_dicts.md +++ b/docs/en/query_language/dicts/external_dicts.md @@ -1,6 +1,4 @@ - - -# External Dictionaries +# External Dictionaries {#dicts-external_dicts} You can add your own dictionaries from various data sources. The data source for a dictionary can be a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see "[Sources for external dictionaries](external_dicts_dict_sources.md)". diff --git a/docs/en/query_language/dicts/external_dicts_dict.md b/docs/en/query_language/dicts/external_dicts_dict.md index bbfe6d82a72..05bb1eac5cf 100644 --- a/docs/en/query_language/dicts/external_dicts_dict.md +++ b/docs/en/query_language/dicts/external_dicts_dict.md @@ -1,6 +1,4 @@ - - -# Configuring an External Dictionary +# Configuring an External Dictionary {#dicts-external_dicts_dict} The dictionary configuration has the following structure: diff --git a/docs/en/query_language/dicts/external_dicts_dict_layout.md b/docs/en/query_language/dicts/external_dicts_dict_layout.md index 6b53004c70f..e3b6498c9a9 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/en/query_language/dicts/external_dicts_dict_layout.md @@ -1,10 +1,8 @@ - - -# Storing Dictionaries in Memory +# Storing Dictionaries in Memory {#dicts-external_dicts_dict_layout} There are a variety of ways to store dictionaries in memory. -We recommend [flat](#flat), [hashed](#hashed) and [complex_key_hashed](#complex-key-hashed). which provide optimal processing speed. +We recommend [flat](#flat), [hashed](#dicts-external_dicts_dict_layout-hashed) and [complex_key_hashed](#complex-key-hashed). which provide optimal processing speed. Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more in the section "[cache](#cache)". @@ -40,7 +38,7 @@ The configuration looks like this: ## Ways to Store Dictionaries in Memory - [flat](#flat) -- [hashed](#hashed) +- [hashed](#dicts-external_dicts_dict_layout-hashed) - [cache](#cache) - [range_hashed](#range-hashed) - [complex_key_hashed](#complex-key-hashed) @@ -65,9 +63,7 @@ Configuration example: ``` - - -### hashed +### hashed {#dicts-external_dicts_dict_layout-hashed} The dictionary is completely stored in memory in the form of a hash table. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. diff --git a/docs/en/query_language/dicts/external_dicts_dict_sources.md b/docs/en/query_language/dicts/external_dicts_dict_sources.md index 44007e87df0..f26967c2d0f 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/en/query_language/dicts/external_dicts_dict_sources.md @@ -1,433 +1,424 @@ - - -# Sources of External Dictionaries - -An external dictionary can be connected from many different sources. - -The configuration looks like this: - -```xml - - - ... - - - - - - ... - - ... - -``` - -The source is configured in the `source` section. - -Types of sources (`source_type`): - -- [Local file](#local-file) -- [Executable file](#executable-file) -- [HTTP(s)](#http-s) -- DBMS - - [MySQL](#mysql) - - [ClickHouse](#clickhouse) - - [MongoDB](#mongodb) - - [ODBC](#odbc) - - - -## Local File - -Example of settings: - -```xml - - - /opt/dictionaries/os.tsv - TabSeparated - - -``` - -Setting fields: - -- `path` – The absolute path to the file. -- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. - - - -## Executable File - -Working with executable files depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file's `STDIN`. - -Example of settings: - -```xml - - - cat /opt/dictionaries/os.tsv - TabSeparated - - -``` - -Setting fields: - -- `command` – The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). -- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. - - - -## HTTP(s) - -Working with an HTTP(s) server depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. - -Example of settings: - -```xml - - - http://[::1]/os.tsv - TabSeparated - - -``` - -In order for ClickHouse to access an HTTPS resource, you must [configure openSSL](../../operations/server_settings/settings.md) in the server configuration. - -Setting fields: - -- `url` – The source URL. -- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. - - - -## ODBC - -You can use this method to connect any database that has an ODBC driver. - -Example of settings: - -```xml - - DatabaseName - ShemaName.TableName
- DSN=some_parameters - SQL_QUERY -
-``` - -Setting fields: - -- `db` – Name of the database. Omit it if the database name is set in the `` parameters. -- `table` – Name of the table and schema if exists. -- `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md). - -ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it's necessary to set table name accordingly to table name case in database. - -### Known vulnerability of the ODBC dictionary functionality - -!!! attention - When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. - -**Example of insecure use** - -Let's configure unixODBC for PostgreSQL. Content of `/etc/odbc.ini`: - -``` -[gregtest] -Driver = /usr/lib/psqlodbca.so -Servername = localhost -PORT = 5432 -DATABASE = test_db -#OPTION = 3 -USERNAME = test -PASSWORD = test -``` - -If you then make a query such as - -``` -SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); -``` - -ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. - -### Example of Connecting PostgreSQL - -Ubuntu OS. - -Installing unixODBC and the ODBC driver for PostgreSQL: - - sudo apt-get install -y unixodbc odbcinst odbc-postgresql - -Configuring `/etc/odbc.ini` (or `~/.odbc.ini`): - -``` - [DEFAULT] - Driver = myconnection - - [myconnection] - Description = PostgreSQL connection to my_db - Driver = PostgreSQL Unicode - Database = my_db - Servername = 127.0.0.1 - UserName = username - Password = password - Port = 5432 - Protocol = 9.3 - ReadOnly = No - RowVersioning = No - ShowSystemTables = No - ConnSettings = -``` - -The dictionary configuration in ClickHouse: - -```xml - - - table_name - - - - - DSN=myconnection - postgresql_table
-
- - - 300 - 360 - - - - - - - id - - - some_column - UInt64 - 0 - - -
-
-``` - -You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. - -### Example of Connecting MS SQL Server - -Ubuntu OS. - -Installing the driver: : - -``` - sudo apt-get install tdsodbc freetds-bin sqsh -``` - -Configuring the driver: : - -``` - $ cat /etc/freetds/freetds.conf - ... - - [MSSQL] - host = 192.168.56.101 - port = 1433 - tds version = 7.0 - client charset = UTF-8 - - $ cat /etc/odbcinst.ini - ... - - [FreeTDS] - Description = FreeTDS - Driver = /usr/lib/x86_64-linux-gnu/odbc/libtdsodbc.so - Setup = /usr/lib/x86_64-linux-gnu/odbc/libtdsS.so - FileUsage = 1 - UsageCount = 5 - - $ cat ~/.odbc.ini - ... - - [MSSQL] - Description = FreeTDS - Driver = FreeTDS - Servername = MSSQL - Database = test - UID = test - PWD = test - Port = 1433 -``` - -Configuring the dictionary in ClickHouse: - -```xml - - - test - - - dict
- DSN=MSSQL;UID=test;PWD=test -
- - - - 300 - 360 - - - - - - - - - k - - - s - String - - - -
-
-``` - -## DBMS - - - -### MySQL - -Example of settings: - -```xml - - - 3306 - clickhouse - qwerty - - example01-1 - 1 - - - example01-2 - 1 - - db_name - table_name
- id=10 - SQL_QUERY -
- -``` - -Setting fields: - -- `port` – The port on the MySQL server. You can specify it for all replicas, or for each one individually (inside ``). - -- `user` – Name of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `password` – Password of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). - -- `replica` – Section of replica configurations. There can be multiple sections. - - `replica/host` – The MySQL host. - - \* `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. - -- `db` – Name of the database. - -- `table` – Name of the table. - -- `where ` – The selection criteria. Optional parameter. - -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md). - -MySQL can be connected on a local host via sockets. To do this, set `host` and `socket`. - -Example of settings: - -```xml - - - localhost - /path/to/socket/file.sock - clickhouse - qwerty - db_name - table_name
- id=10 - SQL_QUERY -
- -``` - - - -### ClickHouse - -Example of settings: - -```xml - - - example01-01-1 - 9000 - default - - default - ids
- id=10 -
- -``` - -Setting fields: - -- `host` – The ClickHouse host. If it is a local host, the query is processed without any network activity. To improve fault tolerance, you can create a [Distributed](../../operations/table_engines/distributed.md) table and enter it in subsequent configurations. -- `port` – The port on the ClickHouse server. -- `user` – Name of the ClickHouse user. -- `password` – Password of the ClickHouse user. -- `db` – Name of the database. -- `table` – Name of the table. -- `where ` – The selection criteria. May be omitted. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md). - - - -### MongoDB - -Example of settings: - -```xml - - - localhost - 27017 - - - test - dictionary_source - - -``` - -Setting fields: - -- `host` – The MongoDB host. -- `port` – The port on the MongoDB server. -- `user` – Name of the MongoDB user. -- `password` – Password of the MongoDB user. -- `db` – Name of the database. -- `collection` – Name of the collection. - -[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_sources/) +# Sources of External Dictionaries {#dicts-external_dicts_dict_sources} + +An external dictionary can be connected from many different sources. + +The configuration looks like this: + +```xml + + + ... + + + + + + ... + + ... + +``` + +The source is configured in the `source` section. + +Types of sources (`source_type`): + +- [Local file](#dicts-external_dicts_dict_sources-local_file) +- [Executable file](#dicts-external_dicts_dict_sources-executable) +- [HTTP(s)](#dicts-external_dicts_dict_sources-http) +- DBMS + - [MySQL](#dicts-external_dicts_dict_sources-mysql) + - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) + - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) + - [ODBC](#dicts-external_dicts_dict_sources-odbc) + + +## Local File {#dicts-external_dicts_dict_sources-local_file} + +Example of settings: + +```xml + + + /opt/dictionaries/os.tsv + TabSeparated + + +``` + +Setting fields: + +- `path` – The absolute path to the file. +- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. + + +## Executable File {#dicts-external_dicts_dict_sources-executable} + +Working with executable files depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file's `STDIN`. + +Example of settings: + +```xml + + + cat /opt/dictionaries/os.tsv + TabSeparated + + +``` + +Setting fields: + +- `command` – The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). +- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. + + +## HTTP(s) {#dicts-external_dicts_dict_sources-http} + +Working with an HTTP(s) server depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. + +Example of settings: + +```xml + + + http://[::1]/os.tsv + TabSeparated + + +``` + +In order for ClickHouse to access an HTTPS resource, you must [configure openSSL](../../operations/server_settings/settings.md) in the server configuration. + +Setting fields: + +- `url` – The source URL. +- `format` – The file format. All the formats described in "[Formats](../../interfaces/formats.md#formats)" are supported. + + +## ODBC {#dicts-external_dicts_dict_sources-odbc} + +You can use this method to connect any database that has an ODBC driver. + +Example of settings: + +```xml + + DatabaseName + ShemaName.TableName
+ DSN=some_parameters + SQL_QUERY +
+``` + +Setting fields: + +- `db` – Name of the database. Omit it if the database name is set in the `` parameters. +- `table` – Name of the table and schema if exists. +- `connection_string` – Connection string. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md). + +ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it's necessary to set table name accordingly to table name case in database. + +### Known vulnerability of the ODBC dictionary functionality + +!!! attention + When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. + +**Example of insecure use** + +Let's configure unixODBC for PostgreSQL. Content of `/etc/odbc.ini`: + +``` +[gregtest] +Driver = /usr/lib/psqlodbca.so +Servername = localhost +PORT = 5432 +DATABASE = test_db +#OPTION = 3 +USERNAME = test +PASSWORD = test +``` + +If you then make a query such as + +``` +SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); +``` + +ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. + +### Example of Connecting PostgreSQL + +Ubuntu OS. + +Installing unixODBC and the ODBC driver for PostgreSQL: + + sudo apt-get install -y unixodbc odbcinst odbc-postgresql + +Configuring `/etc/odbc.ini` (or `~/.odbc.ini`): + +``` + [DEFAULT] + Driver = myconnection + + [myconnection] + Description = PostgreSQL connection to my_db + Driver = PostgreSQL Unicode + Database = my_db + Servername = 127.0.0.1 + UserName = username + Password = password + Port = 5432 + Protocol = 9.3 + ReadOnly = No + RowVersioning = No + ShowSystemTables = No + ConnSettings = +``` + +The dictionary configuration in ClickHouse: + +```xml + + + table_name + + + + + DSN=myconnection + postgresql_table
+
+ + + 300 + 360 + + + + + + + id + + + some_column + UInt64 + 0 + + +
+
+``` + +You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. + +### Example of Connecting MS SQL Server + +Ubuntu OS. + +Installing the driver: : + +``` + sudo apt-get install tdsodbc freetds-bin sqsh +``` + +Configuring the driver: : + +``` + $ cat /etc/freetds/freetds.conf + ... + + [MSSQL] + host = 192.168.56.101 + port = 1433 + tds version = 7.0 + client charset = UTF-8 + + $ cat /etc/odbcinst.ini + ... + + [FreeTDS] + Description = FreeTDS + Driver = /usr/lib/x86_64-linux-gnu/odbc/libtdsodbc.so + Setup = /usr/lib/x86_64-linux-gnu/odbc/libtdsS.so + FileUsage = 1 + UsageCount = 5 + + $ cat ~/.odbc.ini + ... + + [MSSQL] + Description = FreeTDS + Driver = FreeTDS + Servername = MSSQL + Database = test + UID = test + PWD = test + Port = 1433 +``` + +Configuring the dictionary in ClickHouse: + +```xml + + + test + + + dict
+ DSN=MSSQL;UID=test;PWD=test +
+ + + + 300 + 360 + + + + + + + + + k + + + s + String + + + +
+
+``` + +## DBMS + + +### MySQL {#dicts-external_dicts_dict_sources-mysql} + +Example of settings: + +```xml + + + 3306 + clickhouse + qwerty + + example01-1 + 1 + + + example01-2 + 1 + + db_name + table_name
+ id=10 + SQL_QUERY +
+ +``` + +Setting fields: + +- `port` – The port on the MySQL server. You can specify it for all replicas, or for each one individually (inside ``). + +- `user` – Name of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). + +- `password` – Password of the MySQL user. You can specify it for all replicas, or for each one individually (inside ``). + +- `replica` – Section of replica configurations. There can be multiple sections. + - `replica/host` – The MySQL host. + + \* `replica/priority` – The replica priority. When attempting to connect, ClickHouse traverses the replicas in order of priority. The lower the number, the higher the priority. + +- `db` – Name of the database. + +- `table` – Name of the table. + +- `where ` – The selection criteria. Optional parameter. + +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md). + +MySQL can be connected on a local host via sockets. To do this, set `host` and `socket`. + +Example of settings: + +```xml + + + localhost + /path/to/socket/file.sock + clickhouse + qwerty + db_name + table_name
+ id=10 + SQL_QUERY +
+ +``` + + +### ClickHouse {#dicts-external_dicts_dict_sources-clickhouse} + +Example of settings: + +```xml + + + example01-01-1 + 9000 + default + + default + ids
+ id=10 +
+ +``` + +Setting fields: + +- `host` – The ClickHouse host. If it is a local host, the query is processed without any network activity. To improve fault tolerance, you can create a [Distributed](../../operations/table_engines/distributed.md) table and enter it in subsequent configurations. +- `port` – The port on the ClickHouse server. +- `user` – Name of the ClickHouse user. +- `password` – Password of the ClickHouse user. +- `db` – Name of the database. +- `table` – Name of the table. +- `where ` – The selection criteria. May be omitted. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](external_dicts_dict_lifetime.md). + + +### MongoDB {#dicts-external_dicts_dict_sources-mongodb} + +Example of settings: + +```xml + + + localhost + 27017 + + + test + dictionary_source + + +``` + +Setting fields: + +- `host` – The MongoDB host. +- `port` – The port on the MongoDB server. +- `user` – Name of the MongoDB user. +- `password` – Password of the MongoDB user. +- `db` – Name of the database. +- `collection` – Name of the collection. + +[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_sources/) diff --git a/docs/en/query_language/dicts/internal_dicts.md b/docs/en/query_language/dicts/internal_dicts.md index 2b38ffe61f1..627719ea05a 100644 --- a/docs/en/query_language/dicts/internal_dicts.md +++ b/docs/en/query_language/dicts/internal_dicts.md @@ -1,6 +1,4 @@ - - -# Internal dictionaries +# Internal dictionaries {#internal_dicts} ClickHouse contains a built-in feature for working with a geobase. diff --git a/docs/en/query_language/functions/array_join.md b/docs/en/query_language/functions/array_join.md index 01b1d383e3f..ede5b5e9d41 100644 --- a/docs/en/query_language/functions/array_join.md +++ b/docs/en/query_language/functions/array_join.md @@ -1,6 +1,4 @@ - - -# arrayJoin function +# arrayJoin function {#functions_arrayjoin} This is a very unusual function. diff --git a/docs/en/query_language/functions/ext_dict_functions.md b/docs/en/query_language/functions/ext_dict_functions.md index d7b40ff6d6d..6d0a8bb2615 100644 --- a/docs/en/query_language/functions/ext_dict_functions.md +++ b/docs/en/query_language/functions/ext_dict_functions.md @@ -1,6 +1,4 @@ - - -# Functions for working with external dictionaries +# Functions for working with external dictionaries {#ext_dict_functions} For information on connecting and configuring external dictionaries, see "[External dictionaries](../dicts/external_dicts.md)". diff --git a/docs/en/query_language/functions/in_functions.md b/docs/en/query_language/functions/in_functions.md index be43498c5a4..e7b355bb76c 100644 --- a/docs/en/query_language/functions/in_functions.md +++ b/docs/en/query_language/functions/in_functions.md @@ -2,7 +2,7 @@ ## in, notIn, globalIn, globalNotIn -See the section [IN operators](../select.md/). +See the section [IN operators](../select.md#select-in-operators). ## tuple(x, y, ...), operator (x, y, ...) diff --git a/docs/en/query_language/index.md b/docs/en/query_language/index.md index 3f0ba2c8a51..2d457fdaad3 100644 --- a/docs/en/query_language/index.md +++ b/docs/en/query_language/index.md @@ -1,6 +1,6 @@ # SQL Reference -* [SELECT](select.md#select) +* [SELECT](select.md) * [INSERT INTO](insert_into.md) * [CREATE](create.md) * [ALTER](alter.md#query_language_queries_alter) diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index 159a7611206..01cf2c934c2 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -47,7 +47,7 @@ To avoid data loss use the [MergeTree](../operations/table_engines/mergetree.md) If the table is corrupted, you can copy the non-corrupted data to another table. To do this: 1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE AS `. -2. Set the [max_threads](../operations/settings/settings.md#max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. +2. Set the [max_threads](../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. 3. Execute the query `INSERT INTO SELECT * FROM `. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied. 4. Restart the `clickhouse-client` to reset the `max_threads` value. @@ -248,7 +248,7 @@ TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] Removes all data from a table. When the clause `IF EXISTS` is omitted, the query returns an error if the table does not exist. -The `TRUNCATE` query is not supported for [View](../operations/table_engines/view.md#table_engines-view), [File](../operations/table_engines/file.md#table_engines-file), [URL](../operations/table_engines/url.md#table_engines-url) and [Null](../operations/table_engines/null.md) table engines. +The `TRUNCATE` query is not supported for [View](../operations/table_engines/view.md), [File](../operations/table_engines/file.md), [URL](../operations/table_engines/url.md) and [Null](../operations/table_engines/null.md) table engines. ## USE diff --git a/docs/en/query_language/operators.md b/docs/en/query_language/operators.md index 70312613ba1..fc4d152bf40 100644 --- a/docs/en/query_language/operators.md +++ b/docs/en/query_language/operators.md @@ -53,7 +53,7 @@ Groups of operators are listed in order of priority (the higher it is in the lis ## Operators for Working With Data Sets -*See the section [IN operators](select.md#in-operators).* +*See the section [IN operators](select.md#select-in-operators).* `a IN ...` – The `in(a, b) function` @@ -83,9 +83,7 @@ Note: The conditional operator calculates the values of b and c, then checks whether condition a is met, and then returns the corresponding value. If `b` or `C` is an [arrayJoin()](functions/array_join.md#functions_arrayjoin) function, each row will be replicated regardless of the "a" condition. - - -## Conditional Expression +## Conditional Expression {#operator_case} ``` sql CASE [x] @@ -130,9 +128,7 @@ For efficiency, the `and` and `or` functions accept any number of arguments. The ClickHouse supports the `IS NULL` and `IS NOT NULL` operators. - - -### IS NULL +### IS NULL {#operator-is-null} - For [Nullable](../data_types/nullable.md) type values, the `IS NULL` operator returns: - `1`, if the value is `NULL`. diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index ee9ebb49047..02a5cc3578d 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -44,7 +44,7 @@ If a query does not list any columns (for example, SELECT count() FROM t), some The FINAL modifier can be used only for a SELECT from a CollapsingMergeTree table. When you specify FINAL, data is selected fully "collapsed". Keep in mind that using FINAL leads to a selection that includes columns related to the primary key, in addition to the columns specified in the SELECT. Additionally, the query will be executed in a single stream, and data will be merged during query execution. This means that when using FINAL, the query is processed more slowly. In most cases, you should avoid using FINAL. For more information, see the section "CollapsingMergeTree engine". -### SAMPLE Clause +### SAMPLE Clause {#select-sample-clause} The SAMPLE clause allows for approximated query processing. Approximated query processing is only supported by MergeTree\* type tables, and only if the sampling expression was specified during table creation (see the section "MergeTree engine"). @@ -81,7 +81,7 @@ A sample with a relative coefficient is "consistent": if we look at all possible For example, a sample of user IDs takes rows with the same subset of all the possible user IDs from different tables. This allows using the sample in subqueries in the IN clause, as well as for manually correlating results of different queries with samples. -### ARRAY JOIN Clause +### ARRAY JOIN Clause {#select-array-join-clause} Allows executing JOIN with an array or nested data structure. The intent is similar to the 'arrayJoin' function, but its functionality is broader. @@ -339,7 +339,7 @@ The corresponding conversion can be performed before the WHERE/PREWHERE clause ( Joins the data in the usual [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) sense. !!! info "Note" - Not related to [ARRAY JOIN](#array-join). + Not related to [ARRAY JOIN](#select-array-join-clause). ``` sql @@ -374,7 +374,7 @@ When using a normal `JOIN`, the query is sent to remote servers. Subqueries are When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calculate the right table. This temporary table is passed to each remote server, and queries are run on them using the temporary data that was transmitted. -Be careful when using `GLOBAL`. For more information, see the section [Distributed subqueries](#distributed-subqueries). +Be careful when using `GLOBAL`. For more information, see the section [Distributed subqueries](#select-distributed-subqueries). **Usage Recommendations** @@ -443,7 +443,7 @@ If you need a `JOIN` for joining with dimension tables (these are relatively sma The JOIN behavior is affected by the [join_use_nulls](../operations/settings/settings.md) setting. With `join_use_nulls=1`, `JOIN` works like in standard SQL. -If the JOIN keys are [Nullable](../data_types/nullable.md#data_types-nullable) fields, the rows where at least one of the keys has the value [NULL](syntax.md) are not joined. +If the JOIN keys are [Nullable](../data_types/nullable.md) fields, the rows where at least one of the keys has the value [NULL](syntax.md) are not joined. ### WHERE Clause @@ -747,7 +747,7 @@ If the FORMAT clause is omitted, the default format is used, which depends on bo When using the command-line client, data is passed to the client in an internal efficient format. The client independently interprets the FORMAT clause of the query and formats the data itself (thus relieving the network and the server from the load). -### IN Operators +### IN Operators {#select-in-operators} The `IN`, `NOT IN`, `GLOBAL IN`, and `GLOBAL NOT IN` operators are covered separately, since their functionality is quite rich. @@ -845,7 +845,7 @@ FROM t_null ``` -#### Distributed Subqueries +#### Distributed Subqueries {#select-distributed-subqueries} There are two options for IN-s with subqueries (similar to JOINs): normal `IN` / `JOIN` and `GLOBAL IN` / `GLOBAL JOIN`. They differ in how they are run for distributed query processing. diff --git a/docs/en/query_language/syntax.md b/docs/en/query_language/syntax.md index 923ae676ef2..d7307a64376 100644 --- a/docs/en/query_language/syntax.md +++ b/docs/en/query_language/syntax.md @@ -71,9 +71,7 @@ For more information, see the section "Operators2". An array must consist of at least one item, and a tuple must have at least two items. Tuples have a special purpose for use in the IN clause of a SELECT query. Tuples can be obtained as the result of a query, but they can't be saved to a database (with the exception of Memory-type tables). - - -### NULL Literal +### NULL Literal {#null-literal} Indicates that the value is missing. diff --git a/docs/en/query_language/table_functions/file.md b/docs/en/query_language/table_functions/file.md index 22c22bf5b3c..ff9ba31cb28 100644 --- a/docs/en/query_language/table_functions/file.md +++ b/docs/en/query_language/table_functions/file.md @@ -9,7 +9,7 @@ file(path, format, structure) **Input parameters** -- `path` — The relative path to the file from [user_files_path](../../operations/server_settings/settings.md#user_files_path). +- `path` — The relative path to the file from [user_files_path](../../operations/server_settings/settings.md#server_settings-user_files_path). - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format `'colunmn1_name column1_ype, column2_name column2_type, ...'`. diff --git a/docs/fa/data_types/index.md b/docs/fa/data_types/index.md index 5ffabb4344c..c6073522c5c 100644 --- a/docs/fa/data_types/index.md +++ b/docs/fa/data_types/index.md @@ -1,8 +1,6 @@ - -
-# Data types +# Data types {#data_types} ClickHouse قابلیت ذخیره سازی انواع type های مختلف برای ذخیره داده ها در جداول را دارا می باشد. diff --git a/docs/fa/interfaces/cli.md b/docs/fa/interfaces/cli.md index b4c0bc7e0de..4b30ecbfb1b 100644 --- a/docs/fa/interfaces/cli.md +++ b/docs/fa/interfaces/cli.md @@ -70,9 +70,7 @@ command line برا پایه 'readline' (و 'history' یا 'libedit'، یه بد کلاینت commant-line اجازه ی پاس دادن داده های external (جداول موقت external) را برای query ها می دهد. برای اطلاعات بیشتر به بخش "داده های External برای پردازش query" مراجعه کنید. - - -## پیکربندی +## پیکربندی {#interfaces_cli_configuration} شما میتوانید، پارامتر ها را به `clickhouse-client` (تمام پارامترها دارای مقدار پیش فرض هستند) از دو روش زیر پاس بدید: diff --git a/docs/fa/interfaces/formats.md b/docs/fa/interfaces/formats.md index 37257453d50..04e4a4ea3bf 100644 --- a/docs/fa/interfaces/formats.md +++ b/docs/fa/interfaces/formats.md @@ -1,8 +1,6 @@ - -
-# فرمت های Input و Output +# فرمت های Input و Output {#formats} فرمت تعیین می کند که چگونه داده ها پس از اجرای SELECT (چگونه نوشته شده و چگونه توسط سرور فرمت شده) به شما بر می گردد، و چگونه آن برای INSERT ها پذیرفته شده (چگونه آن توسط سرور پارس و خوانده می شود). @@ -144,9 +142,7 @@ SearchPhrase=baku count()=1000 پارس کردن، اجازه می دهد که فیلد اضافه ی `tskv` بدون علامت و مقدار وجود داشته باشد. این فیلد نادیده گرفته می شود. - - -## CSV +## CSV {#csv} Comma Separated Values format ([RFC](https://tools.ietf.org/html/rfc4180)). @@ -160,7 +156,7 @@ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMA
-*به صورت پیش فرض — `,`. برای اطلاعات بیشتر [format_csv_delimiter](/operations/settings/settings/#format_csv_delimiter) را ببینید. +*به صورت پیش فرض — `,`. برای اطلاعات بیشتر [format_csv_delimiter](/operations/settings/settings/#settings-format_csv_delimiter) را ببینید. در هنگام پارس کردن، تمامی مقادیر می توانند با کوتیشن یا بدون کوتیشن پارس شوند. تک کوتیشن و دابل کوتیشن پشتیبانی می شود. سطر ها می توانند بدون کوتیشن تنظیم شوند. در این مورد سطر ها، جدا کننده ها با (CR یا LF) پارس می شوند. در موارد نقض RFC، در هنگام پارس کردن سطر ها بدون کوتیشن، فضاها و tab های پیشین نادید گرفته می شوند. برای line feed، یونیکس از (LF)، ویدنوز از (CR LF) و Mac OS کلاسیک (CR LF) پشتیبانی می کند. @@ -548,9 +544,7 @@ test: string with \'quotes\' and \t with some special \n characters آرایه ها به شکل `HelloWorld...` و tuple ها به صورت `HelloWorld...` در خروجی می آیند. - - -## CapnProto +## CapnProto {#capnproto} Cap'n Proto یک فرمت پیام باینری شبیه به Protocol Buffer و Thrift می باشد، اما شبیه به JSON یا MessagePack نیست. diff --git a/docs/fa/interfaces/index.md b/docs/fa/interfaces/index.md index 0c8914939a6..a3e2eb55a5e 100644 --- a/docs/fa/interfaces/index.md +++ b/docs/fa/interfaces/index.md @@ -1,8 +1,6 @@ - -
-# رابط ها +# رابط ها {#interfaces} ClickHouse دو اینترفیس شبکه را فراهم می کند (هر دو می توانند به صورت اختیاری در TLS برای امنیت اضافی پیچیده شوند): diff --git a/docs/fa/interfaces/third-party/integrations.md b/docs/fa/interfaces/third-party/integrations.md index 88aa4e80935..e0258730e14 100644 --- a/docs/fa/interfaces/third-party/integrations.md +++ b/docs/fa/interfaces/third-party/integrations.md @@ -49,7 +49,7 @@ - [RClickhouse](https://github.com/IMSMWU/RClickhouse) (استفاده می کند [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) - Java - [Hadoop](http://hadoop.apache.org) - - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (استفاده می کند [JDBC](../jdbc.md)) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (استفاده می کند [JDBC](../../query_language/table_functions/jdbc.md)) - Scala - [Akka](https://akka.io) - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) diff --git a/docs/ru/data_types/array.md b/docs/ru/data_types/array.md index d1864c8e2d8..890e314be99 100644 --- a/docs/ru/data_types/array.md +++ b/docs/ru/data_types/array.md @@ -1,6 +1,4 @@ - - -# Array(T) +# Array(T) {#data_type-array} Массив из элементов типа `T`. diff --git a/docs/ru/data_types/date.md b/docs/ru/data_types/date.md index 887bb19e967..8e747eeb554 100644 --- a/docs/ru/data_types/date.md +++ b/docs/ru/data_types/date.md @@ -1,6 +1,4 @@ - - -# Date +# Date {#data_type-date} Дата. Хранится в двух байтах в виде (беззнакового) числа дней, прошедших от 1970-01-01. Позволяет хранить значения от чуть больше, чем начала unix-эпохи до верхнего порога, определяющегося константой на этапе компиляции (сейчас - до 2106 года, последний полностью поддерживаемый год - 2105). Минимальное значение выводится как 0000-00-00. diff --git a/docs/ru/data_types/datetime.md b/docs/ru/data_types/datetime.md index d9f5457e3ac..19d701ddad7 100644 --- a/docs/ru/data_types/datetime.md +++ b/docs/ru/data_types/datetime.md @@ -1,6 +1,4 @@ - - -# DateTime +# DateTime {#data_type-datetime} Дата-с-временем. Хранится в 4 байтах, в виде (беззнакового) unix timestamp. Позволяет хранить значения в том же интервале, что и для типа Date. Минимальное значение выводится как 0000-00-00 00:00:00. Время хранится с точностью до одной секунды (без учёта секунд координации). diff --git a/docs/ru/data_types/index.md b/docs/ru/data_types/index.md index a167ec3859f..e6c06152b9c 100644 --- a/docs/ru/data_types/index.md +++ b/docs/ru/data_types/index.md @@ -1,6 +1,4 @@ - - -# Типы данных +# Типы данных {#data_types} ClickHouse может сохранять в ячейках таблиц данные различных типов. diff --git a/docs/ru/data_types/nested_data_structures/aggregatefunction.md b/docs/ru/data_types/nested_data_structures/aggregatefunction.md index 99bc4d9932e..6e4cabc098a 100644 --- a/docs/ru/data_types/nested_data_structures/aggregatefunction.md +++ b/docs/ru/data_types/nested_data_structures/aggregatefunction.md @@ -1,6 +1,4 @@ - - -# AggregateFunction(name, types_of_arguments...) +# AggregateFunction(name, types_of_arguments...) {#data_type-aggregatefunction} Промежуточное состояние агрегатной функции. Чтобы его получить, используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. diff --git a/docs/ru/data_types/nullable.md b/docs/ru/data_types/nullable.md index 0899e4b896c..5bc6350280a 100644 --- a/docs/ru/data_types/nullable.md +++ b/docs/ru/data_types/nullable.md @@ -1,6 +1,4 @@ - - -# Nullable(TypeName) +# Nullable(TypeName) {#data_type-nullable} Позволяет работать как со значением типа `TypeName` так и с отсутствием этого значения ([NULL](../query_language/syntax.md)) в одной и той же переменной, в том числе хранить `NULL` в таблицах вместе со значения типа `TypeName`. Например, в столбце типа `Nullable(Int8)` можно хранить значения типа `Int8`, а в тех строках, где значения нет, будет храниться `NULL`. diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index ec7bc7d2c1b..8a0d954445a 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -68,10 +68,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA Клиент командной строки позволяет передать внешние данные (внешние временные таблицы) для использования запроса. Подробнее смотрите раздел "Внешние данные для обработки запроса" - - - -## Конфигурирование +## Конфигурирование {#interfaces_cli_configuration} В `clickhouse-client` можно передавать различные параметры (все параметры имеют значения по умолчанию) с помощью: diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index d09d53391df..659f3327d18 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1,6 +1,4 @@ - - -# Форматы входных и выходных данных +# Форматы входных и выходных данных {#formats} ClickHouse может принимать (`INSERT`) и отдавать (`SELECT`) данные в различных форматах. @@ -32,9 +30,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT [XML](#xml) | ✗ | ✔ | [CapnProto](#capnproto) | ✔ | ✔ | - - -## TabSeparated +## TabSeparated {#tabseparated} В TabSeparated формате данные пишутся по строкам. Каждая строчка содержит значения, разделённые табами. После каждого значения идёт таб, кроме последнего значения в строке, после которого идёт перевод строки. Везде подразумеваются исключительно unix-переводы строк. Последняя строка также обязана содержать перевод строки на конце. Значения пишутся в текстовом виде, без обрамляющих кавычек, с экранированием служебных символов. @@ -97,34 +93,29 @@ world [NULL](../query_language/syntax.md) форматируется в виде `\N`. - - -## TabSeparatedRaw +## TabSeparatedRaw {#tabseparatedraw} Отличается от формата `TabSeparated` тем, что строки выводятся без экранирования. Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). Этот формат также доступен под именем `TSVRaw`. - -## TabSeparatedWithNames +## TabSeparatedWithNames {#tabseparatedwithnames} Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов. При парсинге, первая строка полностью игнорируется: вы не можете использовать имена столбцов, чтобы указать их порядок расположения, или чтобы проверить их корректность. (Поддержка обработки заголовка при парсинге может быть добавлена в будущем.) Этот формат также доступен под именем `TSVWithNames`. - -## TabSeparatedWithNamesAndTypes +## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов, а во второй - типы столбцов. При парсинге, первая и вторая строка полностью игнорируется. Этот формат также доступен под именем `TSVWithNamesAndTypes`. - -## TSKV +## TSKV {#tskv} Похож на TabSeparated, но выводит значения в формате name=value. Имена экранируются так же, как строки в формате TabSeparated и, дополнительно, экранируется также символ =. @@ -156,19 +147,17 @@ x=1 y=\N При парсинге, в качестве дополнительного поля, может присутствовать `tskv` без знака равенства и без значения. Это поле игнорируется. - - -## CSV +## CSV {#csv} Формат comma separated values ([RFC](https://tools.ietf.org/html/rfc4180)). -При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются символом-разделителем, по умолчанию — `,`. Символ-разделитель определяется настройкой [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter). Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж). +При форматировании, строки выводятся в двойных кавычках. Двойная кавычка внутри строки выводится как две двойные кавычки подряд. Других правил экранирования нет. Даты и даты-с-временем выводятся в двойных кавычках. Числа выводятся без кавычек. Значения разделяются символом-разделителем, по умолчанию — `,`. Символ-разделитель определяется настройкой [format_csv_delimiter](../operations/settings/settings.md#settings-format_csv_delimiter). Строки разделяются unix переводом строки (LF). Массивы сериализуются в CSV следующим образом: сначала массив сериализуется в строку, как в формате TabSeparated, а затем полученная строка выводится в CSV в двойных кавычках. Кортежи в формате CSV сериализуются, как отдельные столбцы (то есть, теряется их вложенность в кортеж). ``` clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv ``` -*По умолчанию — `,`. См. настройку [format_csv_delimiter](/operations/settings/settings/#format_csv_delimiter) для дополнительной информации. +*По умолчанию — `,`. См. настройку [format_csv_delimiter](/operations/settings/settings/#settings-format_csv_delimiter) для дополнительной информации. При парсинге, все значения могут парситься как в кавычках, так и без кавычек. Поддерживаются как двойные, так и одинарные кавычки. В том числе, строки могут быть расположены без кавычек - тогда они парсятся до символа-разделителя или перевода строки (CR или LF). В нарушение RFC, в случае парсинга строк не в кавычках, начальные и конечные пробелы и табы игнорируются. В качестве перевода строки, поддерживаются как Unix (LF), так и Windows (CR LF) и Mac OS Classic (LF CR) варианты. @@ -179,9 +168,8 @@ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMA ## CSVWithNames Выводит также заголовок, аналогично `TabSeparatedWithNames`. - -## JSON +## JSON {#json} Выводит данные в формате JSON. Кроме таблицы с данными, также выводятся имена и типы столбцов, и некоторая дополнительная информация - общее количество выведенных строк, а также количество строк, которое могло бы быть выведено, если бы не было LIMIT-а. Пример: @@ -270,9 +258,7 @@ ClickHouse поддерживает [NULL](../query_language/syntax.md), кот Смотрите также формат JSONEachRow. - - -## JSONCompact +## JSONCompact {#jsoncompact} Отличается от JSON только тем, что строчки данных выводятся в массивах, а не в object-ах. @@ -317,9 +303,8 @@ ClickHouse поддерживает [NULL](../query_language/syntax.md), кот Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). Смотрите также формат `JSONEachRow`. - -## JSONEachRow +## JSONEachRow {#jsoneachrow} Выводит данные в виде отдельных JSON объектов для каждой строки (newline delimited JSON). @@ -339,22 +324,19 @@ ClickHouse поддерживает [NULL](../query_language/syntax.md), кот В отличие от формата JSON, нет замены невалидных UTF-8 последовательностей. В строках может выводиться произвольный набор байт. Это сделано для того, чтобы данные форматировались без потери информации. Экранирование значений осуществляется аналогично формату JSON. При парсинге, поддерживается расположение значений разных столбцов в произвольном порядке. Допустимо отсутствие некоторых значений - тогда они воспринимаются как равные значениям по умолчанию. При этом, в качестве значений по умолчанию используются нули, пустые строки и не поддерживаются сложные значения по умолчанию, которые могут быть заданы в таблице. Пропускаются пробельные символы между элементами. После объектов может быть расположена запятая, которая игнорируется. Объекты не обязательно должны быть разделены переводами строк. - -## Native +## Native {#native} Самый эффективный формат. Данные пишутся и читаются блоками в бинарном виде. Для каждого блока пишется количество строк, количество столбцов, имена и типы столбцов, а затем кусочки столбцов этого блока, один за другим. То есть, этот формат является "столбцовым" - не преобразует столбцы в строки. Именно этот формат используется в родном интерфейсе - при межсерверном взаимодействии, при использовании клиента командной строки, при работе клиентов, написанных на C++. Вы можете использовать этот формат для быстрой генерации дампов, которые могут быть прочитаны только СУБД ClickHouse. Вряд ли имеет смысл работать с этим форматом самостоятельно. - -## Null +## Null {#null} Ничего не выводит. При этом, запрос обрабатывается, а при использовании клиента командной строки, данные ещё и передаются на клиент. Используется для тестов, в том числе, тестов производительности. Очевидно, формат подходит только для вывода, но не для парсинга. - -## Pretty +## Pretty {#pretty} Выводит данные в виде Unicode-art табличек, также используя ANSI-escape последовательности для установки цветов в терминале. Рисуется полная сетка таблицы и, таким образом, каждая строчка занимает две строки в терминале. @@ -403,20 +385,16 @@ Extremes: └────────────┴─────────┘ ``` - - -## PrettyCompact +## PrettyCompact {#prettycompact} Отличается от `Pretty` тем, что не рисуется сетка между строками - результат более компактный. Этот формат используется по умолчанию в клиенте командной строки в интерактивном режиме. - -## PrettyCompactMonoBlock +## PrettyCompactMonoBlock {#prettycompactmonoblock} Отличается от [PrettyCompact](#prettycompact) тем, что строки (до 10 000 штук) буферизуются и затем выводятся в виде одной таблицы, а не по блокам. - -## PrettyNoEscapes +## PrettyNoEscapes {#prettynoescapes} Отличается от Pretty тем, что не используются ANSI-escape последовательности. Это нужно для отображения этого формата в браузере, а также при использовании утилиты командной строки watch. @@ -435,14 +413,12 @@ watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FOR ### PrettySpaceNoEscapes Аналогично. - -## PrettySpace +## PrettySpace {#prettyspace} Отличается от [PrettyCompact](#prettycompact) тем, что вместо сетки используется пустое пространство (пробелы). - -## RowBinary +## RowBinary {#rowbinary} Форматирует и парсит данные по строкам, в бинарном виде. Строки и значения уложены подряд, без разделителей. Формат менее эффективен, чем формат Native, так как является строковым. @@ -464,9 +440,7 @@ Array представлены как длина в формате varint (unsig Именно этот формат используется в запросе `INSERT INTO t VALUES ...`, но вы также можете использовать его для форматирования результатов запросов. - - -## Vertical +## Vertical {#vertical} Выводит каждое значение на отдельной строке, с указанием имени столбца. Формат удобно использовать для вывода одной-нескольких строк, если каждая строка состоит из большого количества столбцов. @@ -486,9 +460,7 @@ y: ᴺᵁᴸᴸ Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). - - -## VerticalRaw +## VerticalRaw {#verticalraw} Отличается от формата `Vertical` тем, что строки выводятся без экранирования. Этот формат подходит только для вывода результата выполнения запроса, но не для парсинга (приёма данных для вставки в таблицу). @@ -514,9 +486,8 @@ Row 1: ────── test: string with \'quotes\' and \t with some special \n characters ``` - -## XML +## XML {#xml} Формат XML подходит только для вывода данных, не для парсинга. Пример: @@ -590,9 +561,7 @@ test: string with \'quotes\' and \t with some special \n characters Массивы выводятся как `HelloWorld...`, а кортежи как `HelloWorld...`. - - -## CapnProto +## CapnProto {#capnproto} Cap'n Proto - формат бинарных сообщений, похож на Protocol Buffers и Thrift, но не похож на JSON или MessagePack. diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 24d59c8e3d6..162eb494166 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -1,6 +1,4 @@ - - -# HTTP-интерфейс +# HTTP-интерфейс {#http_interface} HTTP интерфейс позволяет использовать ClickHouse на любой платформе, из любого языка программирования. У нас он используется для работы из Java и Perl, а также из shell-скриптов. В других отделах, HTTP интерфейс используется из Perl, Python и Go. HTTP интерфейс более ограничен по сравнению с родным интерфейсом, но является более совместимым. diff --git a/docs/ru/interfaces/index.md b/docs/ru/interfaces/index.md index bc3b1f2bd79..e655d198d1c 100644 --- a/docs/ru/interfaces/index.md +++ b/docs/ru/interfaces/index.md @@ -1,6 +1,4 @@ - - -# Интерфейсы +# Интерфейсы {#interfaces} ClickHouse предоставляет два сетевых интерфейса (оба могут быть дополнительно обернуты в TLS для дополнительной безопасности): diff --git a/docs/ru/interfaces/third-party/integrations.md b/docs/ru/interfaces/third-party/integrations.md index d32fd2a888c..0d7b92e26d3 100644 --- a/docs/ru/interfaces/third-party/integrations.md +++ b/docs/ru/interfaces/third-party/integrations.md @@ -46,7 +46,7 @@ - [RClickhouse](https://github.com/IMSMWU/RClickhouse) (использует [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) - Java - [Hadoop](http://hadoop.apache.org) - - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (использует [JDBC](../jdbc.md)) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (использует [JDBC](../../query_language/table_functions/jdbc.md)) - Scala - [Akka](https://akka.io) - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) diff --git a/docs/ru/operations/access_rights.md b/docs/ru/operations/access_rights.md index e9a8160ae5c..a55c9e0e798 100644 --- a/docs/ru/operations/access_rights.md +++ b/docs/ru/operations/access_rights.md @@ -96,6 +96,6 @@ Пользователь может получить список всех БД и таблиц в них с помощью запросов `SHOW` или системных таблиц, даже если у него нет доступа к отдельным БД. -Доступ к БД не связан с настройкой [readonly](settings/query_complexity.md#readonly). Невозможно дать полный доступ к одной БД и `readonly` к другой. +Доступ к БД не связан с настройкой [readonly](settings/permissions_for_queries.md#settings_readonly). Невозможно дать полный доступ к одной БД и `readonly` к другой. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/access_rights/) diff --git a/docs/ru/operations/configuration_files.md b/docs/ru/operations/configuration_files.md index 00bb4865f36..ebb95652d61 100644 --- a/docs/ru/operations/configuration_files.md +++ b/docs/ru/operations/configuration_files.md @@ -1,6 +1,4 @@ - - -# Конфигурационные файлы +# Конфигурационные файлы {#configuration_files} Основной конфигурационный файл сервера - `config.xml`. Он расположен в директории `/etc/clickhouse-server/`. diff --git a/docs/ru/operations/quotas.md b/docs/ru/operations/quotas.md index 93a795973d9..57f1ad17b11 100644 --- a/docs/ru/operations/quotas.md +++ b/docs/ru/operations/quotas.md @@ -1,6 +1,4 @@ - - -# Квоты +# Квоты {#quotas} Квоты позволяют ограничить использование ресурсов за некоторый интервал времени, или просто подсчитывать использование ресурсов. Квоты настраиваются в конфиге пользователей. Обычно это users.xml. diff --git a/docs/ru/operations/server_settings/index.md b/docs/ru/operations/server_settings/index.md index c208f9b418b..b7a67bacaf9 100644 --- a/docs/ru/operations/server_settings/index.md +++ b/docs/ru/operations/server_settings/index.md @@ -1,6 +1,4 @@ - - -# Конфигурационные параметры сервера +# Конфигурационные параметры сервера {#server_settings} Раздел содержит описания настроек сервера, которые не могут изменяться на уровне сессии или запроса. diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 4996a283f8d..9c855fa6ff0 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -120,7 +120,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ## format_schema_path -Путь к каталогу со схемами для входных данных. Например со схемами для формата [CapnProto](../../interfaces/formats.md#format_capnproto). +Путь к каталогу со схемами для входных данных. Например со схемами для формата [CapnProto](../../interfaces/formats.md#capnproto). **Пример** @@ -222,9 +222,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ``` - - -## include_from +## include_from {#server_settings-include_from} Путь к файлу с подстановками. @@ -643,7 +641,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat 8589934592 ``` -## user_files_path +## user_files_path {#server_settings-user_files_path} Каталог с пользовательскими файлами. Используется в табличной функции [file()](../../query_language/table_functions/file.md). diff --git a/docs/ru/operations/settings/index.md b/docs/ru/operations/settings/index.md index 1fa2f55bd53..b9b211e9394 100644 --- a/docs/ru/operations/settings/index.md +++ b/docs/ru/operations/settings/index.md @@ -1,6 +1,4 @@ - - -# Настройки +# Настройки {#settings} Все настройки, описанные ниже, могут быть заданы несколькими способами. Настройки задаются послойно, т.е. каждый следующий слой перезаписывает предыдущие настройки. diff --git a/docs/ru/operations/settings/permissions_for_queries.md b/docs/ru/operations/settings/permissions_for_queries.md index 120f019c4ca..69a00ab4f8e 100644 --- a/docs/ru/operations/settings/permissions_for_queries.md +++ b/docs/ru/operations/settings/permissions_for_queries.md @@ -1,6 +1,4 @@ - - -# Разрешения для запросов +# Разрешения для запросов {#permissions_for_queries} Запросы в ClickHouse можно разделить на несколько типов: @@ -17,9 +15,7 @@ `KILL QUERY` выполняется с любыми настройками. - - -## readonly +## readonly {#settings_readonly} Ограничивает разрешения для запросов на чтение данных, запись данных и изменение параметров. @@ -39,9 +35,7 @@ 0 - - -## allow_ddl +## allow_ddl {#settings_allow_ddl} Разрешает/запрещает [DDL](https://ru.wikipedia.org/wiki/Data_Definition_Language) запросы. diff --git a/docs/ru/operations/settings/query_complexity.md b/docs/ru/operations/settings/query_complexity.md index 2300e506324..b67bf76d8ba 100644 --- a/docs/ru/operations/settings/query_complexity.md +++ b/docs/ru/operations/settings/query_complexity.md @@ -17,9 +17,7 @@ `any (только для group_by_overflow_mode)` - продолжить агрегацию по ключам, которые успели войти в набор, но не добавлять новые ключи в набор. - - -## max_memory_usage +## max_memory_usage {#settings_max_memory_usage} Максимальный возможный объем оперативной памяти для выполнения запроса на одном сервере. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 60d82582678..a06290f51f1 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -32,9 +32,7 @@ ClickHouse применяет настройку в тех случаях, ко По умолчанию - 1 (включена). - - -## force_index_by_date +## force_index_by_date {#settings-force_index_by_date} Запрещает выполнение запросов, если использовать индекс по дате невозможно. @@ -108,9 +106,7 @@ ClickHouse применяет настройку в тех случаях, ко log_queries=1 - - -## max_insert_block_size +## max_insert_block_size {#settings-max_insert_block_size} Формировать блоки указанного размера, при вставке в таблицу. Эта настройка действует только в тех случаях, когда сервер сам формирует такие блоки. @@ -122,9 +118,7 @@ ClickHouse применяет настройку в тех случаях, ко Это намного больше, чем `max_block_size`. Это сделано, потому что некоторые движки таблиц (`*MergeTree`) будут на каждый вставляемый блок формировать кусок данных на диске, что является довольно большой сущностью. Также, в таблицах типа `*MergeTree`, данные сортируются при вставке, и достаточно большой размер блока позволяет отсортировать больше данных в оперативке. - - -## max_replica_delay_for_distributed_queries +## max_replica_delay_for_distributed_queries {#settings_settings_max_replica_delay_for_distributed_queries} Отключает отстающие реплики при распределенных запросах. Смотрите "[Репликация](../../operations/table_engines/replication.md)". @@ -134,7 +128,7 @@ ClickHouse применяет настройку в тех случаях, ко Используется при выполнении `SELECT` из распределенной таблицы, которая указывает на реплицированные таблицы. -## max_threads +## max_threads {#settings-max_threads} Максимальное количество потоков обработки запроса - без учёта потоков для чтения данных с удалённых серверов (смотрите параметр max_distributed_connections). @@ -256,14 +250,13 @@ ClickHouse применяет настройку в тех случаях, ко ## stream_flush_interval_ms -Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max_insert_block_size](#max-insert-block-size) строк. +Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max_insert_block_size](#settings-max_insert_block_size) строк. Значение по умолчанию - 7500. Чем меньше значение, тем чаще данные сбрасываются в таблицу. Установка слишком низкого значения приводит к снижению производительности. - ## load_balancing На какие реплики (среди живых реплик) предпочитать отправлять запрос (при первой попытке) при распределённой обработке запроса. @@ -333,9 +326,7 @@ ClickHouse применяет настройку в тех случаях, ко Если значение истинно, то при использовании JSON\* форматов UInt64 и Int64 числа выводятся в кавычках (из соображений совместимости с большинством реализаций JavaScript), иначе - без кавычек. - - -## format_csv_delimiter +## format_csv_delimiter {#settings-format_csv_delimiter} Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`. @@ -347,7 +338,7 @@ ClickHouse применяет настройку в тех случаях, ко При `join_use_nulls=1` `JOIN` ведёт себя как в стандартном SQL, т.е. если при слиянии возникают пустые ячейки, то тип соответствующего поля преобразуется к [Nullable](../../data_types/nullable.md#data_type-nullable), а пустые ячейки заполняются значениями [NULL](../../query_language/syntax.md). -## insert_quorum +## insert_quorum {#settings-insert_quorum} Включает кворумную запись. @@ -362,7 +353,7 @@ ClickHouse применяет настройку в тех случаях, ко Все реплики в кворуме консистентны, т.е. содержат данные всех более ранних запросов `INSERT`. Последовательность `INSERT` линеаризуется. -При чтении данных, записанных с `insert_quorum` можно использовать настройку [select_sequential_consistency](#select-sequential-consistency). +При чтении данных, записанных с `insert_quorum` можно использовать настройку [select_sequential_consistency](#settings-select_sequential_consistency). **ClickHouse генерирует исключение** @@ -371,11 +362,11 @@ ClickHouse применяет настройку в тех случаях, ко **См. также параметры** -- [insert_quorum_timeout](#insert-quorum-timeout) -- [select_sequential_consistency](#select-sequential-consitency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert_quorum_timeout +## insert_quorum_timeout {#settings-insert_quorum_timeout} Время ожидания кворумной записи в секундах. Если время прошло, а запись так не состоялась, то ClickHouse сгенерирует исключение и клиент должен повторить запрос на запись того же блока на эту же или любую другую реплику. @@ -383,11 +374,11 @@ ClickHouse применяет настройку в тех случаях, ко **См. также параметры** -- [insert_quorum](#insert-quorum) -- [select_sequential_consistency](#select-sequential-consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select_sequential_consistency +## select_sequential_consistency {#settings-select_sequential_consistency} Включение/выключение последовательной консистентности для запросов `SELECT`: @@ -398,7 +389,7 @@ ClickHouse применяет настройку в тех случаях, ко См. также параметры: -- [insert_quorum](#insert-quorum) -- [insert_quorum_timeout](#insert-quorum-timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/settings/settings/) diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index d283a8dbe04..2a1d7f0d111 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -1,6 +1,4 @@ - - -# CollapsingMergeTree +# CollapsingMergeTree {#table_engine-collapsingmergetree} *Движок достаточно специфичен для Яндекс.Метрики.* diff --git a/docs/ru/operations/table_engines/file.md b/docs/ru/operations/table_engines/file.md index 0c920e66edb..731204f928a 100644 --- a/docs/ru/operations/table_engines/file.md +++ b/docs/ru/operations/table_engines/file.md @@ -1,6 +1,4 @@ - - -# File(Format) +# File(Format) {#table_engines-file} Управляет данными в одном файле на диске в указанном формате. diff --git a/docs/ru/operations/table_engines/index.md b/docs/ru/operations/table_engines/index.md index a5e42c21edb..cf38d90b55f 100644 --- a/docs/ru/operations/table_engines/index.md +++ b/docs/ru/operations/table_engines/index.md @@ -1,6 +1,4 @@ - - -# Движки таблиц +# Движки таблиц {#table_engines} Движок таблицы (тип таблицы) определяет: diff --git a/docs/ru/operations/table_engines/kafka.md b/docs/ru/operations/table_engines/kafka.md index f747967cf19..73b4fb395ee 100644 --- a/docs/ru/operations/table_engines/kafka.md +++ b/docs/ru/operations/table_engines/kafka.md @@ -105,7 +105,7 @@ Kafka SETTINGS SELECT level, sum(total) FROM daily GROUP BY level; ``` -Для улучшения производительности полученные сообщения группируются в блоки размера [max_insert_block_size](../settings/settings.md#settings-settings-max_insert_block_size). Если блок не удалось сформировать за [stream_flush_interval_ms](../settings/settings.md) миллисекунд, то данные будут сброшены в таблицу независимо от полноты блока. +Для улучшения производительности полученные сообщения группируются в блоки размера [max_insert_block_size](../settings/settings.md#settings-max_insert_block_size). Если блок не удалось сформировать за [stream_flush_interval_ms](../settings/settings.md) миллисекунд, то данные будут сброшены в таблицу независимо от полноты блока. Чтобы остановить получение данных топика или изменить логику преобразования, отсоедините материализованное представление: diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index 2828a893fe5..f1f65b94442 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -1,6 +1,4 @@ - - -# MergeTree +# MergeTree {#table_engines-mergetree} Движок `MergeTree`, а также другие движки этого семейства (`*MergeTree`) — это наиболее функциональные движки таблиц ClickHousе. @@ -222,7 +220,7 @@ ClickHouse будет использовать индекс по первичн SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -Чтобы проверить, сможет ли ClickHouse использовать индекс при выполнении запроса, используйте настройки [force_index_by_date](../settings/settings.md#settings-settings-force_index_by_date) и [force_primary_key](../settings/settings.md). +Чтобы проверить, сможет ли ClickHouse использовать индекс при выполнении запроса, используйте настройки [force_index_by_date](../settings/settings.md#settings-force_index_by_date) и [force_primary_key](../settings/settings.md). Ключ партиционирования по месяцам обеспечивает чтение только тех блоков данных, которые содержат даты из нужного диапазона. При этом блок данных может содержать данные за многие даты (до целого месяца). В пределах одного блока данные упорядочены по первичному ключу, который может не содержать дату в качестве первого столбца. В связи с этим, при использовании запроса с указанием условия только на дату, но не на префикс первичного ключа, будет читаться данных больше, чем за одну дату. diff --git a/docs/ru/operations/table_engines/replication.md b/docs/ru/operations/table_engines/replication.md index e2925dc3783..7eb9e1216c3 100644 --- a/docs/ru/operations/table_engines/replication.md +++ b/docs/ru/operations/table_engines/replication.md @@ -1,6 +1,4 @@ - - -# Репликация данных +# Репликация данных {#table_engines-replication} Репликация поддерживается только для таблиц семейства MergeTree: diff --git a/docs/ru/operations/table_engines/summingmergetree.md b/docs/ru/operations/table_engines/summingmergetree.md index 59dd4c64a1f..b10d1598fb8 100644 --- a/docs/ru/operations/table_engines/summingmergetree.md +++ b/docs/ru/operations/table_engines/summingmergetree.md @@ -89,7 +89,7 @@ SELECT key, sum(value) FROM summtt GROUP BY key При вставке данных в таблицу они сохраняются как есть. Периодически ClickHouse выполняет слияние вставленных кусков данных и именно в этот момент производится суммирование и замена многих строк с одинаковым первичным ключом на одну для каждого результирующего куска данных. -ClickHouse может слить куски данных таким образом, что не все строки с одинаковым первичным ключом окажутся в одном финальном куске, т.е. суммирование будет не полным. Поэтому, при выборке данных (`SELECT`) необходимо использовать агрегатную функцию [sum()](../../query_language/agg_functions/reference.md#sum) и секцию `GROUP BY` как описано в примере выше. +ClickHouse может слить куски данных таким образом, что не все строки с одинаковым первичным ключом окажутся в одном финальном куске, т.е. суммирование будет не полным. Поэтому, при выборке данных (`SELECT`) необходимо использовать агрегатную функцию [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) и секцию `GROUP BY` как описано в примере выше. ### Общие правила суммирования diff --git a/docs/ru/operations/table_engines/url.md b/docs/ru/operations/table_engines/url.md index 35d9a5170b3..f67242ee870 100644 --- a/docs/ru/operations/table_engines/url.md +++ b/docs/ru/operations/table_engines/url.md @@ -1,6 +1,4 @@ - - -# URL(URL, Format) +# URL(URL, Format) {#table_engines-url} Управляет данными на удаленном HTTP/HTTPS сервере. Данный движок похож на движок [File](file.md). diff --git a/docs/ru/operations/utils/index.md b/docs/ru/operations/utils/index.md index 3e71f2a3d28..544fa1827f7 100644 --- a/docs/ru/operations/utils/index.md +++ b/docs/ru/operations/utils/index.md @@ -1,6 +1,6 @@ # Утилиты ClickHouse * [clickhouse-local](clickhouse-local.md* [clickhouse-copier](clickhouse-copier.md) данные с одного кластера на другой. -* [clickhouse-copier](clickhouse-copier.md#utils-clickhouse-copier) - копирует (и перешардирует) данные с одного кластера на другой. +* [clickhouse-copier](clickhouse-copier.md) - копирует (и перешардирует) данные с одного кластера на другой. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/utils/) diff --git a/docs/ru/query_language/agg_functions/combinators.md b/docs/ru/query_language/agg_functions/combinators.md index b823d6b877c..dfee76bb79d 100644 --- a/docs/ru/query_language/agg_functions/combinators.md +++ b/docs/ru/query_language/agg_functions/combinators.md @@ -1,12 +1,8 @@ - - -# Комбинаторы агрегатных функций +# Комбинаторы агрегатных функций {#aggregate_functions_combinators} К имени агрегатной функции может быть приписан некоторый суффикс. При этом, работа агрегатной функции некоторым образом модифицируется. - - -## -If +## -If {#agg-functions-combinator-if} К имени любой агрегатной функции может быть приписан суффикс -If. В этом случае, агрегатная функция принимает ещё один дополнительный аргумент - условие (типа UInt8). Агрегатная функция будет обрабатывать только те строки, для которых условие сработало. Если условие ни разу не сработало - возвращается некоторое значение по умолчанию (обычно - нули, пустые строки). diff --git a/docs/ru/query_language/agg_functions/index.md b/docs/ru/query_language/agg_functions/index.md index e89934fde39..5bcd7fa0301 100644 --- a/docs/ru/query_language/agg_functions/index.md +++ b/docs/ru/query_language/agg_functions/index.md @@ -1,6 +1,4 @@ - - -# Агрегатные функции +# Агрегатные функции {#aggregate_functions} Агрегатные функции работают в [привычном](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) для специалистов по базам данных смысле. diff --git a/docs/ru/query_language/agg_functions/parametric_functions.md b/docs/ru/query_language/agg_functions/parametric_functions.md index 999aff99679..119d8c32053 100644 --- a/docs/ru/query_language/agg_functions/parametric_functions.md +++ b/docs/ru/query_language/agg_functions/parametric_functions.md @@ -1,6 +1,4 @@ - - -# Параметрические агрегатные функции +# Параметрические агрегатные функции {#aggregate_functions_parametric} Некоторые агрегатные функции могут принимать не только столбцы-аргументы (по которым производится свёртка), но и набор параметров - констант для инициализации. Синтаксис - две пары круглых скобок вместо одной. Первая - для параметров, вторая - для аргументов. diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index 2b440de9f27..dafbcc07a5c 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -8,9 +8,7 @@ Запрос вида `SELECT count() FROM table` не оптимизируется, так как количество записей в таблице нигде не хранится отдельно. Из таблицы будет выбран какой-нибудь достаточно маленький столбец, и будет посчитано количество значений в нём. - - -## any(x) +## any(x) {#agg_function-any} Выбирает первое попавшееся значение. Порядок выполнения запроса может быть произвольным и даже каждый раз разным, поэтому результат данной функции недетерминирован. @@ -20,7 +18,7 @@ При наличии в запросе `SELECT` секции `GROUP BY` или хотя бы одной агрегатной функции, ClickHouse (в отличие от, например, MySQL) требует, чтобы все выражения в секциях `SELECT`, `HAVING`, `ORDER BY` вычислялись из ключей или из агрегатных функций. То есть, каждый выбираемый из таблицы столбец, должен использоваться либо в ключах, либо внутри агрегатных функций. Чтобы получить поведение, как в MySQL, вы можете поместить остальные столбцы в агрегатную функцию `any`. -## anyHeavy(x) +## anyHeavy(x) {#agg_function-anyHeavy} Выбирает часто встречающееся значение с помощью алгоритма "[heavy hitters](http://www.cs.umd.edu/~samir/498/karp.pdf)". Если существует значение, которое встречается чаще, чем в половине случаев, в каждом потоке выполнения запроса, то возвращается данное значение. В общем случае, результат недетерминирован. @@ -47,7 +45,7 @@ FROM ontime └───────┘ ``` -## anyLast(x) +## anyLast(x) {#agg_function-anyLast} Выбирает последнее попавшееся значение. Результат так же недетерминирован, как и для функции `any`. @@ -214,12 +212,12 @@ SELECT argMin(user, salary) FROM salary Вычисляет значение arg при максимальном значении val. Если есть несколько разных значений arg для максимальных значений val, то выдаётся первое попавшееся из таких значений. -## sum(x) +## sum(x) {#agg_function-sum} Вычисляет сумму. Работает только для чисел. -## sumWithOverflow(x) +## sumWithOverflow(x) {#agg_function-sumWithOverflow} Вычисляет сумму чисел, используя для результата тот же тип данных, что и для входных параметров. Если сумма выйдет за максимальное значение для заданного типа данных, то функция вернёт ошибку. @@ -268,9 +266,7 @@ GROUP BY timeslot Работает только для чисел. Результат всегда Float64. - - -## uniq(x) +## uniq(x) {#agg_function-uniq} Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей. diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index 9eaed2dae51..945338d3126 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -1,6 +1,4 @@ - - -## ALTER +## ALTER {#query_language_queries_alter} Запрос `ALTER` поддерживается только для таблиц типа `*MergeTree`, а также `Merge` и `Distributed`. Запрос имеет несколько вариантов. ### Манипуляции со столбцами @@ -227,9 +225,7 @@ ALTER TABLE [db.]table FETCH PARTITION 'name' FROM 'path-in-zookeeper' Для запросов `ALTER ... ATTACH|DETACH|DROP` можно настроить ожидание, с помощью настройки `replication_alter_partitions_sync`. Возможные значения: `0` - не ждать, `1` - ждать выполнения только у себя (по умолчанию), `2` - ждать всех. - - -### Мутации +### Мутации {#query_language_queries_show_databases} Мутации - разновидность запроса ALTER, позволяющая изменять или удалять данные в таблице. В отличие от стандартных запросов `DELETE` и `UPDATE`, рассчитанных на точечное изменение данных, область применения мутаций - достаточно тяжёлые изменения, затрагивающие много строк в таблице. diff --git a/docs/ru/query_language/dicts/external_dicts.md b/docs/ru/query_language/dicts/external_dicts.md index c598298744b..d03b33942ef 100644 --- a/docs/ru/query_language/dicts/external_dicts.md +++ b/docs/ru/query_language/dicts/external_dicts.md @@ -1,6 +1,4 @@ - - -# Внешние словари +# Внешние словари {#dicts-external_dicts} Существует возможность подключать собственные словари из различных источников данных. Источником данных для словаря может быть локальный текстовый/исполняемый файл, HTTP(s) ресурс или другая СУБД. Подробнее смотрите в разделе "[Источники внешних словарей](external_dicts_dict_sources.md)". diff --git a/docs/ru/query_language/dicts/external_dicts_dict.md b/docs/ru/query_language/dicts/external_dicts_dict.md index aa4d8a3a78d..776ffb24756 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict.md +++ b/docs/ru/query_language/dicts/external_dicts_dict.md @@ -1,6 +1,4 @@ - - -# Настройка внешнего словаря +# Настройка внешнего словаря {#dicts-external_dicts_dict} Конфигурация словаря имеет следующую структуру: diff --git a/docs/ru/query_language/dicts/external_dicts_dict_layout.md b/docs/ru/query_language/dicts/external_dicts_dict_layout.md index 26bfa7ac564..fbac880c403 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_layout.md @@ -1,6 +1,4 @@ - - -# Хранение словарей в памяти +# Хранение словарей в памяти {#dicts-external_dicts_dict_layout} Словари можно размещать в памяти множеством способов. @@ -65,8 +63,6 @@ ``` - - ### hashed Словарь полностью хранится в оперативной памяти в виде хэш-таблиц. Словарь может содержать произвольное количество элементов с произвольными идентификаторами. На практике, количество ключей может достигать десятков миллионов элементов. @@ -82,7 +78,7 @@ ``` -### complex_key_hashed +### complex_key_hashed Тип размещения предназначен для использования с составными [ключами](external_dicts_dict_structure.md). Аналогичен `hashed`. diff --git a/docs/ru/query_language/dicts/internal_dicts.md b/docs/ru/query_language/dicts/internal_dicts.md index da4e9846e55..2cd55d71f99 100644 --- a/docs/ru/query_language/dicts/internal_dicts.md +++ b/docs/ru/query_language/dicts/internal_dicts.md @@ -1,6 +1,4 @@ - - -# Встроенные словари +# Встроенные словари {#internal_dicts} ClickHouse содержит встроенную возможность работы с геобазой. diff --git a/docs/ru/query_language/functions/array_join.md b/docs/ru/query_language/functions/array_join.md index 57bdbe94c75..7a89ab73fb4 100644 --- a/docs/ru/query_language/functions/array_join.md +++ b/docs/ru/query_language/functions/array_join.md @@ -1,6 +1,4 @@ - - -# Функция arrayJoin +# Функция arrayJoin {#functions_arrayjoin} Это совсем необычная функция. diff --git a/docs/ru/query_language/functions/ext_dict_functions.md b/docs/ru/query_language/functions/ext_dict_functions.md index 89b23d23671..c166f03c8be 100644 --- a/docs/ru/query_language/functions/ext_dict_functions.md +++ b/docs/ru/query_language/functions/ext_dict_functions.md @@ -1,6 +1,4 @@ - - -# Функции для работы с внешними словарями +# Функции для работы с внешними словарями {#ext_dict_functions} Информация о подключении и настройке внешних словарей смотрите в разделе "[Внешние словари](../dicts/external_dicts.md)". diff --git a/docs/ru/query_language/functions/higher_order_functions.md b/docs/ru/query_language/functions/higher_order_functions.md index e9cb1d0de17..e26546f11df 100644 --- a/docs/ru/query_language/functions/higher_order_functions.md +++ b/docs/ru/query_language/functions/higher_order_functions.md @@ -1,6 +1,4 @@ - - -# Функции высшего порядка +# Функции высшего порядка {#higher_order_functions} ## Оператор `->`, функция lambda(params, expr) diff --git a/docs/ru/query_language/functions/in_functions.md b/docs/ru/query_language/functions/in_functions.md index 0663f5252e9..7eb87e53a6e 100644 --- a/docs/ru/query_language/functions/in_functions.md +++ b/docs/ru/query_language/functions/in_functions.md @@ -1,7 +1,7 @@ # Функции для реализации оператора IN. ## in, notIn, globalIn, globalNotIn -Смотрите раздел [Операторы IN](../select.md#operatori-in). +Смотрите раздел [Операторы IN](../select.md#select-in-operators). ## tuple(x, y, ...), оператор (x, y, ...) Функция, позволяющая сгруппировать несколько столбцов. diff --git a/docs/ru/query_language/functions/ym_dict_functions.md b/docs/ru/query_language/functions/ym_dict_functions.md index 7d37e1b0c87..c086d7404c8 100644 --- a/docs/ru/query_language/functions/ym_dict_functions.md +++ b/docs/ru/query_language/functions/ym_dict_functions.md @@ -1,6 +1,4 @@ - - -# Функции для работы со словарями Яндекс.Метрики +# Функции для работы со словарями Яндекс.Метрики {#ym_dict_functions} Чтобы указанные ниже функции работали, в конфиге сервера должны быть указаны пути и адреса для получения всех словарей Яндекс.Метрики. Словари загружаются при первом вызове любой из этих функций. Если справочники не удаётся загрузить - будет выкинуто исключение. diff --git a/docs/ru/query_language/index.md b/docs/ru/query_language/index.md index 6c919518e1f..881a6ef7cfc 100644 --- a/docs/ru/query_language/index.md +++ b/docs/ru/query_language/index.md @@ -1,6 +1,6 @@ # Справка по SQL -* [SELECT](select.md#select) +* [SELECT](select.md) * [INSERT INTO](insert_into.md) * [CREATE](create.md) * [ALTER](alter.md#query_language_queries_alter) diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 3c177fe44c9..000daf746dc 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -1,7 +1,4 @@ - - - -## ATTACH +## ATTACH {#queries} Запрос полностью аналогичен запросу `CREATE`, но: - вместо слова `CREATE` используется слово `ATTACH`; diff --git a/docs/ru/query_language/operators.md b/docs/ru/query_language/operators.md index 21888259332..edfc2513d28 100644 --- a/docs/ru/query_language/operators.md +++ b/docs/ru/query_language/operators.md @@ -53,7 +53,7 @@ ## Операторы для работы с множествами -*Смотрите раздел [Операторы IN](select.md#operatori-in).* +*Смотрите раздел [Операторы IN](select.md#select-in-operators).* `a IN ...` - функция `in(a, b)` @@ -83,9 +83,7 @@ Условный оператор сначала вычисляет значения b и c, затем проверяет выполнение условия a, и только после этого возвращает соответствующее значение. Если в качестве b или с выступает функция [arrayJoin()](functions/array_join.md#functions_arrayjoin), то размножение каждой строки произойдет вне зависимости от условия а. - - -## Условное выражение +## Условное выражение {#operator_case} ``` sql CASE [x] @@ -128,9 +126,7 @@ P.S. Функция `transform` не умеет работать с `NULL`. ClickHouse поддерживает операторы `IS NULL` и `IS NOT NULL`. - - -### IS NULL +### IS NULL {#operator-is-null} - Для значений типа [Nullable](../data_types/nullable.md) оператор `IS NULL` возвращает: - `1`, если значение — `NULL`. diff --git a/docs/ru/query_language/select.md b/docs/ru/query_language/select.md index 14e84a28d5a..ce1fa6de9a0 100644 --- a/docs/ru/query_language/select.md +++ b/docs/ru/query_language/select.md @@ -436,13 +436,13 @@ LIMIT 10 На поведение JOIN влияет настройка [join_use_nulls](../operations/settings/settings.md). При `join_use_nulls=1` `JOIN` работает как в стандартном SQL. -Если ключами JOIN выступают поля типа [Nullable](../data_types/nullable.md#data_types-nullable), то строки, где хотя бы один из ключей имеет значение [NULL](syntax.md), не соединяются. +Если ключами JOIN выступают поля типа [Nullable](../data_types/nullable.md), то строки, где хотя бы один из ключей имеет значение [NULL](syntax.md), не соединяются. -### Секция WHERE +### Секция WHERE {#select-where} -Позволяет задать выражение, которое ClickHouse использует для фильтрации данных перед всеми другими действиями в запросе кроме выражений, содержащихся в секции [PREWHERE](#prewhere). Обычно, это выражение с логическими операторами. +Позволяет задать выражение, которое ClickHouse использует для фильтрации данных перед всеми другими действиями в запросе кроме выражений, содержащихся в секции [PREWHERE](#select-prewhere). Обычно, это выражение с логическими операторами. Результат выражения должен иметь тип `UInt8`. @@ -467,9 +467,9 @@ WHERE isNull(y) ``` -### Секция PREWHERE +### Секция PREWHERE {#select-prewhere} -Имеет такой же смысл, как и секция [WHERE](#where). Отличие состоит в том, какие данные читаются из таблицы. +Имеет такой же смысл, как и секция [WHERE](#select-where). Отличие состоит в том, какие данные читаются из таблицы. При использовании `PREWHERE`, из таблицы сначала читаются только столбцы, необходимые для выполнения `PREWHERE`. Затем читаются остальные столбцы, нужные для выполнения запроса, но из них только те блоки, в которых выражение в `PREWHERE` истинное. `PREWHERE` имеет смысл использовать, если есть условия фильтрации, которые использует меньшинство столбцов из тех, что есть в запросе, но достаточно сильно фильтрует данные. Таким образом, сокращается количество читаемых данных. @@ -762,7 +762,7 @@ SELECT CounterID, 2 AS table, sum(Sign) AS c При использовании клиента командной строки данные на клиент передаются во внутреннем эффективном формате. При этом клиент самостоятельно интерпретирует секцию FORMAT запроса и форматирует данные на своей стороне (снимая нагрузку на сеть и сервер). -### Операторы IN +### Операторы IN {#select-in-operators} Операторы `IN`, `NOT IN`, `GLOBAL IN`, `GLOBAL NOT IN` рассматриваются отдельно, так как их функциональность достаточно богатая. diff --git a/docs/ru/query_language/syntax.md b/docs/ru/query_language/syntax.md index f2008fd8260..e5548e61b33 100644 --- a/docs/ru/query_language/syntax.md +++ b/docs/ru/query_language/syntax.md @@ -67,9 +67,7 @@ INSERT INTO t VALUES (1, 'Hello, world'), (2, 'abc'), (3, 'def') Массив должен состоять хотя бы из одного элемента, а кортеж - хотя бы из двух. Кортежи носят служебное значение для использования в секции IN запроса SELECT. Кортежи могут быть получены в качестве результата запроса, но не могут быть сохранены в базу (за исключением таблиц типа Memory). - - -### NULL +### NULL {#null-literal} Обозначает, что значение отсутствует. diff --git a/docs/ru/query_language/table_functions/file.md b/docs/ru/query_language/table_functions/file.md index 816643de1ba..0fb16bf5a48 100644 --- a/docs/ru/query_language/table_functions/file.md +++ b/docs/ru/query_language/table_functions/file.md @@ -9,7 +9,7 @@ file(path, format, structure) **Входные параметры** -- `path` — относительный путь до файла от [user_files_path](../../operations/server_settings/settings.md#user_files_path). +- `path` — относительный путь до файла от [user_files_path](../../operations/server_settings/settings.md#server_settings-user_files_path). - `format` — [формат](../../interfaces/formats.md#formats) файла. - `structure` — структура таблицы. Формат `'colunmn1_name column1_ype, column2_name column2_type, ...'`. diff --git a/docs/zh/data_types/array.md b/docs/zh/data_types/array.md index 845b91b4158..b6764498183 100644 --- a/docs/zh/data_types/array.md +++ b/docs/zh/data_types/array.md @@ -1,6 +1,4 @@ - - -# Array(T) +# Array(T) {#data_type-array} 由 `T` 类型元素组成的数组。 diff --git a/docs/zh/data_types/datetime.md b/docs/zh/data_types/datetime.md index 6988d7da283..0bdd942bd72 100644 --- a/docs/zh/data_types/datetime.md +++ b/docs/zh/data_types/datetime.md @@ -1,6 +1,4 @@ - - -# DateTime +# DateTime {#data_type-datetime} 时间戳类型。用四个字节(无符号的)存储 Unix 时间戳)。允许存储与日期类型相同的范围内的值。最小值为 0000-00-00 00:00:00。时间戳类型值精确到秒(没有闰秒)。 diff --git a/docs/zh/data_types/index.md b/docs/zh/data_types/index.md index 42fc574aa2e..70aa976cb11 100644 --- a/docs/zh/data_types/index.md +++ b/docs/zh/data_types/index.md @@ -1,6 +1,4 @@ - - -# 数据类型 +# 数据类型 {#data_types} ClickHouse 可以在数据表中存储多种数据类型。 diff --git a/docs/zh/data_types/nullable.md b/docs/zh/data_types/nullable.md index 53f6e7e9f8b..41565f9d721 100644 --- a/docs/zh/data_types/nullable.md +++ b/docs/zh/data_types/nullable.md @@ -1,10 +1,8 @@ - - -# Nullable(TypeName) +# Nullable(TypeName) {#data_type-nullable} 允许用特殊标记 ([NULL](../query_language/syntax.md)) 表示"缺失值",可以与 `TypeName` 的正常值存放一起。例如,`Nullable(Int8)` 类型的列可以存储 `Int8` 类型值,而没有值的行将存储 `NULL`。 -对于 `TypeName`,不能使用复合数据类型 [Array](array.md#data_type is array) 和 [Tuple](tuple.md)。复合数据类型可以包含 `Nullable` 类型值,例如`Array(Nullable(Int8))`。 +对于 `TypeName`,不能使用复合数据类型 [Array](array.md) 和 [Tuple](tuple.md)。复合数据类型可以包含 `Nullable` 类型值,例如`Array(Nullable(Int8))`。 `Nullable` 类型字段不能包含在表索引中。 diff --git a/docs/zh/data_types/tuple.md b/docs/zh/data_types/tuple.md index 7e81736c82d..c8231c00aa2 100644 --- a/docs/zh/data_types/tuple.md +++ b/docs/zh/data_types/tuple.md @@ -3,7 +3,7 @@ 元组,其中每个元素都有单独的 [类型](index.md#data_types)。 -不能在表中存储元组(除了内存表)。它们可以用于临时列分组。在查询中,IN 表达式和带特定参数的 lambda 函数可以来对临时列进行分组。更多信息,请参阅 [IN 操作符](../query_language/select.md) and [Higher order functions](../query_language/functions/higher_order_functions.md#higher_order_functions)。 +不能在表中存储元组(除了内存表)。它们可以用于临时列分组。在查询中,IN 表达式和带特定参数的 lambda 函数可以来对临时列进行分组。更多信息,请参阅 [IN 操作符](../query_language/select.md) and [Higher order functions](../query_language/functions/higher_order_functions.md)。 元组可以是查询的结果。在这种情况下,对于JSON以外的文本格式,括号中的值是逗号分隔的。在JSON格式中,元组作为数组输出(在方括号中)。 diff --git a/docs/zh/interfaces/cli.md b/docs/zh/interfaces/cli.md index bf207034e4b..7a61a315b2b 100644 --- a/docs/zh/interfaces/cli.md +++ b/docs/zh/interfaces/cli.md @@ -63,9 +63,7 @@ cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMA 命令行客户端允许通过外部数据 (外部临时表) 来查询。更多相关信息,请参考 "[外部数据查询处理](../operations/table_engines/external_data.md)". - - -## 配置 +## 配置 {#interfaces_cli_configuration} 您可以通过以下方式传入参数到 `clickhouse-client` 中 (所有的参数都有默认值): diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index c0a0dea6e81..e10532174ff 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -1,6 +1,4 @@ - - -# 输入输出格式 +# 输入输出格式 {#formats} ClickHouse 可以接受多种数据格式,可以在 (`INSERT`) 以及 (`SELECT`) 请求中使用。 @@ -32,9 +30,7 @@ ClickHouse 可以接受多种数据格式,可以在 (`INSERT`) 以及 (`SELECT | [XML](#xml) | ✗ | ✔ | | [CapnProto](#capnproto) | ✔ | ✔ | - - -## TabSeparated +## TabSeparated {#tabseparated} 在 TabSeparated 格式中,数据按行写入。每行包含由制表符分隔的值。除了行中的最后一个值(后面紧跟换行符)之外,每个值都跟随一个制表符。 在任何地方都可以使用严格的 Unix 命令行。最后一行还必须在最后包含换行符。值以文本格式编写,不包含引号,并且要转义特殊字符。 @@ -97,34 +93,29 @@ world [NULL](../query_language/syntax.md) 将输出为 `\N`。 - - -## TabSeparatedRaw +## TabSeparatedRaw {#tabseparatedraw} 与 `TabSeparated` 格式不一样的是,行数据是不会被转义的。 该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 这种格式也可以使用名称 `TSVRaw` 来表示。 - -## TabSeparatedWithNames +## TabSeparatedWithNames {#tabseparatedwithnames} 与 `TabSeparated` 格式不一样的是,第一行会显示列的名称。 在解析过程中,第一行完全被忽略。您不能使用列名来确定其位置或检查其正确性。 (未来可能会加入解析头行的功能) 这种格式也可以使用名称 ` TSVWithNames` 来表示。 - -## TabSeparatedWithNamesAndTypes +## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} 与 `TabSeparated` 格式不一样的是,第一行会显示列的名称,第二行会显示列的类型。 在解析过程中,第一行和第二行完全被忽略。 这种格式也可以使用名称 ` TSVWithNamesAndTypes` 来表示。 - -## TSKV +## TSKV {#tskv} 与 `TabSeparated` 格式类似,但它输出的是 `name=value` 的格式。名称会和 `TabSeparated` 格式一样被转义,`=` 字符也会被转义。 @@ -156,9 +147,8 @@ x=1 y=\N 数据的输出和解析都支持这种格式。对于解析,任何顺序都支持不同列的值。可以省略某些值,用 `-` 表示, 它们被视为等于它们的默认值。在这种情况下,零和空行被用作默认值。作为默认值,不支持表中指定的复杂值。 对于不带等号或值,可以用附加字段 `tskv` 来表示,这种在解析上是被允许的。这样的话该字段被忽略。 - -## CSV +## CSV {#csv} 按逗号分隔的数据格式([RFC](https://tools.ietf.org/html/rfc4180))。 @@ -180,9 +170,8 @@ CSV 格式是和 TabSeparated 一样的方式输出总数和极值。 ## CSVWithNames 会输出带头部行,和 `TabSeparatedWithNames` 一样。 - -## JSON +## JSON {#json} 以 JSON 格式输出数据。除了数据表之外,它还输出列名称和类型以及一些附加信息:输出行的总数以及在没有 LIMIT 时可以输出的行数。 例: @@ -271,9 +260,7 @@ ClickHouse 支持 [NULL](../query_language/syntax.md), 在 JSON 格式中以 `nu 参考 JSONEachRow 格式。 - - -## JSONCompact +## JSONCompact {#jsoncompact} 与 JSON 格式不同的是它以数组的方式输出结果,而不是以结构体。 @@ -318,9 +305,8 @@ ClickHouse 支持 [NULL](../query_language/syntax.md), 在 JSON 格式中以 `nu 这种格式仅仅适用于输出结果集,而不适用于解析(将数据插入到表中)。 参考 `JSONEachRow` 格式。 - -## JSONEachRow +## JSONEachRow {#jsoneachrow} 将数据结果每一行以 JSON 结构体输出(换行分割 JSON 结构体)。 @@ -340,22 +326,19 @@ ClickHouse 支持 [NULL](../query_language/syntax.md), 在 JSON 格式中以 `nu 与 JSON 格式不同的是,没有替换无效的UTF-8序列。任何一组字节都可以在行中输出。这是必要的,因为这样数据可以被格式化而不会丢失任何信息。值的转义方式与JSON相同。 对于解析,任何顺序都支持不同列的值。可以省略某些值 - 它们被视为等于它们的默认值。在这种情况下,零和空行被用作默认值。 作为默认值,不支持表中指定的复杂值。元素之间的空白字符被忽略。如果在对象之后放置逗号,它将被忽略。对象不一定必须用新行分隔。 - -## Native +## Native {#native} 最高性能的格式。 据通过二进制格式的块进行写入和读取。对于每个块,该块中的行数,列数,列名称和类型以及列的部分将被相继记录。 换句话说,这种格式是 “列式”的 - 它不会将列转换为行。 这是用于在服务器之间进行交互的本地界面中使用的格式,用于使用命令行客户端和 C++ 客户端。 您可以使用此格式快速生成只能由 ClickHouse DBMS 读取的格式。但自己处理这种格式是没有意义的。 - -## Null +## Null {#null} 没有输出。但是,查询已处理完毕,并且在使用命令行客户端时,数据将传输到客户端。这仅用于测试,包括生产力测试。 显然,这种格式只适用于输出,不适用于解析。 - -## Pretty +## Pretty {#pretty} 将数据以表格形式输出,也可以使用 ANSI 转义字符在终端中设置颜色。 它会绘制一个完整的表格,每行数据在终端中占用两行。 @@ -405,19 +388,15 @@ Extremes: └────────────┴─────────┘ ``` - - -## PrettyCompact +## PrettyCompact {#prettycompact} 与 `Pretty` 格式不一样的是,`PrettyCompact` 去掉了行之间的表格分割线,这样使得结果更加紧凑。这种格式会在交互命令行客户端下默认使用。 - -## PrettyCompactMonoBlock +## PrettyCompactMonoBlock {#prettycompactmonoblock} 与 `PrettyCompact` 格式不一样的是,它支持 10,000 行数据缓冲,然后输出在一个表格中,不会按照块来区分 - -## PrettyNoEscapes +## PrettyNoEscapes {#prettynoescapes} 与 `Pretty` 格式不一样的是,它不使用 ANSI 字符转义, 这在浏览器显示数据以及在使用 `watch` 命令行工具是有必要的。 @@ -436,14 +415,12 @@ watch -n1 "clickhouse-client --query='SELECT event, value FROM system.events FOR ### PrettySpaceNoEscapes 用法类似上述。 - -## PrettySpace +## PrettySpace {#prettyspace} 与 `PrettyCompact`(#prettycompact) 格式不一样的是,它使用空格来代替网格来显示数据。 - -## RowBinary +## RowBinary {#rowbinary} 以二进制格式逐行格式化和解析数据。行和值连续列出,没有分隔符。 这种格式比 Native 格式效率低,因为它是基于行的。 @@ -466,9 +443,7 @@ FixedString 被简单地表示为一个字节序列。 这是 `INSERT INTO t VALUES ...` 中可以使用的格式,但您也可以将其用于查询结果。 - - -## Vertical +## Vertical {#vertical} 使用指定的列名在单独的行上打印每个值。如果每行都包含大量列,则此格式便于打印一行或几行。 @@ -489,9 +464,7 @@ y: ᴺᵁᴸᴸ 该格式仅适用于输出查询结果,但不适用于解析输入(将数据插入到表中)。 - - -## VerticalRaw +## VerticalRaw {#verticalraw} 和 `Vertical` 格式不同点在于,行是不会被转义的。 这种格式仅仅适用于输出,但不适用于解析输入(将数据插入到表中)。 @@ -520,9 +493,7 @@ Row 1: test: string with \'quotes\' and \t with some special \n characters ``` - - -## XML +## XML {#xml} 该格式仅适用于输出查询结果,但不适用于解析输入,示例: @@ -596,9 +567,7 @@ test: string with \'quotes\' and \t with some special \n characters 数组输出为 ` Hello World ... `,元组输出为 ` Hello World ... ` 。 - - -## CapnProto +## CapnProto {#capnproto} Cap'n Proto 是一种二进制消息格式,类似 Protocol Buffers 和 Thriftis,但与 JSON 或 MessagePack 格式不一样。 diff --git a/docs/zh/interfaces/index.md b/docs/zh/interfaces/index.md index 5334ee2d400..12b61c3f9fd 100644 --- a/docs/zh/interfaces/index.md +++ b/docs/zh/interfaces/index.md @@ -1,6 +1,4 @@ - - -# 客户端 +# 客户端 {#interfaces} ClickHouse提供了两个网络接口(两者都可以选择包装在TLS中以提高安全性): diff --git a/docs/zh/interfaces/third-party/integrations.md b/docs/zh/interfaces/third-party/integrations.md index f3a2e49bfa5..fb5467f3192 100644 --- a/docs/zh/interfaces/third-party/integrations.md +++ b/docs/zh/interfaces/third-party/integrations.md @@ -43,7 +43,7 @@ - [RClickhouse](https://github.com/IMSMWU/RClickhouse) (使用 [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp)) - Java - [Hadoop](http://hadoop.apache.org) - - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (使用 [JDBC](../jdbc.md)) + - [clickhouse-hdfs-loader](https://github.com/jaykelin/clickhouse-hdfs-loader) (使用 [JDBC](../../query_language/table_functions/jdbc.md)) - Scala - [Akka](https://akka.io) - [clickhouse-scala-client](https://github.com/crobox/clickhouse-scala-client) diff --git a/docs/zh/operations/access_rights.md b/docs/zh/operations/access_rights.md index c72a131b173..621d1673d8f 100644 --- a/docs/zh/operations/access_rights.md +++ b/docs/zh/operations/access_rights.md @@ -96,7 +96,7 @@ Access to the `system` database is always allowed (since this database is used f The user can get a list of all databases and tables in them by using `SHOW` queries or system tables, even if access to individual databases isn't allowed. -Database access is not related to the [readonly](settings/query_complexity.md#readonly) setting. You can't grant full access to one database and `readonly` access to another one. +Database access is not related to the [readonly](settings/permissions_for_queries.md#settings_readonly) setting. You can't grant full access to one database and `readonly` access to another one. [Original article](https://clickhouse.yandex/docs/en/operations/access_rights/) diff --git a/docs/zh/operations/configuration_files.md b/docs/zh/operations/configuration_files.md index 0f7fdd3ae09..acbc37634d8 100644 --- a/docs/zh/operations/configuration_files.md +++ b/docs/zh/operations/configuration_files.md @@ -1,6 +1,4 @@ - - -# Configuration Files +# Configuration Files {#configuration_files} The main server config file is `config.xml`. It resides in the `/etc/clickhouse-server/` directory. diff --git a/docs/zh/operations/quotas.md b/docs/zh/operations/quotas.md index 148cb39458c..5379d242da3 100644 --- a/docs/zh/operations/quotas.md +++ b/docs/zh/operations/quotas.md @@ -1,6 +1,4 @@ - - -# Quotas +# Quotas {#quotas} Quotas allow you to limit resource usage over a period of time, or simply track the use of resources. Quotas are set up in the user config. This is usually 'users.xml'. diff --git a/docs/zh/operations/server_settings/index.md b/docs/zh/operations/server_settings/index.md index 88f11c48f4b..3b721f95ef2 100644 --- a/docs/zh/operations/server_settings/index.md +++ b/docs/zh/operations/server_settings/index.md @@ -1,6 +1,4 @@ - - -# Server configuration parameters +# Server configuration parameters {#server_settings} This section contains descriptions of server settings that cannot be changed at the session or query level. diff --git a/docs/zh/operations/server_settings/settings.md b/docs/zh/operations/server_settings/settings.md index 24a384a9087..0eadd7ebcec 100644 --- a/docs/zh/operations/server_settings/settings.md +++ b/docs/zh/operations/server_settings/settings.md @@ -120,7 +120,7 @@ The default is `true`. ## format_schema_path -The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#format_capnproto) format. +The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../interfaces/formats.md#capnproto) format. **Example** @@ -221,9 +221,7 @@ Opens `https://tabix.io/` when accessing ` http://localhost: http_port`. ``` - - -## include_from +## include_from {#server_settings-include_from} The path to the file with substitutions. @@ -640,7 +638,7 @@ The uncompressed cache is advantageous for very short queries in individual case 8589934592 ``` -## user_files_path +## user_files_path {#server_settings-user_files_path} The directory with user files. Used in the table function [file()](../../query_language/table_functions/file.md). diff --git a/docs/zh/operations/settings/index.md b/docs/zh/operations/settings/index.md index 5676796fd10..73152183079 100644 --- a/docs/zh/operations/settings/index.md +++ b/docs/zh/operations/settings/index.md @@ -1,6 +1,4 @@ - - -# Settings +# Settings {#settings} There are multiple ways to make all the settings described below. Settings are configured in layers, so each subsequent layer redefines the previous settings. diff --git a/docs/zh/operations/settings/query_complexity.md b/docs/zh/operations/settings/query_complexity.md index 9e49dc58ca3..eb8e722e887 100644 --- a/docs/zh/operations/settings/query_complexity.md +++ b/docs/zh/operations/settings/query_complexity.md @@ -14,9 +14,7 @@ It can take one of two values: `throw` or `break`. Restrictions on aggregation ( `any (only for group_by_overflow_mode)` – Continuing aggregation for the keys that got into the set, but don't add new keys to the set. - - -## readonly +## readonly {#query_complexity_readonly} With a value of 0, you can execute any queries. With a value of 1, you can only execute read requests (such as SELECT and SHOW). Requests for writing and changing settings (INSERT, SET) are prohibited. @@ -26,9 +24,7 @@ After enabling readonly mode, you can't disable it in the current session. When using the GET method in the HTTP interface, 'readonly = 1' is set automatically. In other words, for queries that modify data, you can only use the POST method. You can send the query itself either in the POST body, or in the URL parameter. - - -## max_memory_usage +## max_memory_usage {#settings_max_memory_usage} The maximum amount of RAM to use for running a query on a single server. diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index 2e77b8d0af7..3a456749a78 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -32,9 +32,7 @@ Used when performing `SELECT` from a distributed table that points to replicate By default, 1 (enabled). - - -## force_index_by_date +## force_index_by_date {#settings-settings-force_index_by_date} Disables query execution if the index can't be used by date. @@ -108,9 +106,7 @@ Queries sent to ClickHouse with this setup are logged according to the rules in log_queries=1 - - -## max_insert_block_size +## max_insert_block_size {#settings-max_insert_block_size} The size of blocks to form for insertion into a table. This setting only applies in cases when the server forms the blocks. @@ -122,9 +118,7 @@ By default, it is 1,048,576. This is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allows sorting more data in RAM. - - -## max_replica_delay_for_distributed_queries +## max_replica_delay_for_distributed_queries {#settings_settings_max_replica_delay_for_distributed_queries} Disables lagging replicas for distributed queries. See "[Replication](../../operations/table_engines/replication.md)". @@ -134,7 +128,7 @@ Default value: 0 (off). Used when performing `SELECT` from a distributed table that points to replicated tables. -## max_threads +## max_threads {#settings-max_threads} The maximum number of query processing threads @@ -258,7 +252,7 @@ This parameter is useful when you are using formats that require a schema defini ## stream_flush_interval_ms -Works for tables with streaming in the case of a timeout, or when a thread generates[max_insert_block_size](#max-insert-block-size) rows. +Works for tables with streaming in the case of a timeout, or when a thread generates [max_insert_block_size](#settings-max_insert_block_size) rows. The default value is 7500. @@ -337,9 +331,7 @@ It works for JSONEachRow and TSKV formats. If the value is true, integers appear in quotes when using JSON\* Int64 and UInt64 formats (for compatibility with most JavaScript implementations); otherwise, integers are output without the quotes. - - -## format_csv_delimiter +## format_csv_delimiter {#format_csv_delimiter} The character interpreted as a delimiter in the CSV data. By default, the delimiter is `,`. diff --git a/docs/zh/operations/table_engines/collapsingmergetree.md b/docs/zh/operations/table_engines/collapsingmergetree.md index 02fa1a0e5a2..8fd80265168 100644 --- a/docs/zh/operations/table_engines/collapsingmergetree.md +++ b/docs/zh/operations/table_engines/collapsingmergetree.md @@ -1,6 +1,4 @@ - - -# CollapsingMergeTree +# CollapsingMergeTree {#table_engine-collapsingmergetree} The engine inherits from [MergeTree](mergetree.md) and adds the logic of rows collapsing to data parts merge algorithm. diff --git a/docs/zh/operations/table_engines/file.md b/docs/zh/operations/table_engines/file.md index a394db256db..c63d78a2ba3 100644 --- a/docs/zh/operations/table_engines/file.md +++ b/docs/zh/operations/table_engines/file.md @@ -1,6 +1,4 @@ - - -# File(InputFormat) +# File(InputFormat) {#table_engines-file} The data source is a file that stores data in one of the supported input formats (TabSeparated, Native, etc.). diff --git a/docs/zh/operations/table_engines/kafka.md b/docs/zh/operations/table_engines/kafka.md index fc341a041ff..d592c5a958e 100644 --- a/docs/zh/operations/table_engines/kafka.md +++ b/docs/zh/operations/table_engines/kafka.md @@ -105,7 +105,7 @@ Kafka SETTINGS SELECT level, sum(total) FROM daily GROUP BY level; ``` -为了提高性能,接受的消息被分组为 [max_insert_block_size](../settings/settings.md#settings-settings-max_insert_block_size) 大小的块。如果未在 [stream_flush_interval_ms](../settings/settings.md) 毫秒内形成块,则不关心块的完整性,都会将数据刷新到表中。 +为了提高性能,接受的消息被分组为 [max_insert_block_size](../settings/settings.md#settings-max_insert_block_size) 大小的块。如果未在 [stream_flush_interval_ms](../settings/settings.md) 毫秒内形成块,则不关心块的完整性,都会将数据刷新到表中。 停止接收主题数据或更改转换逻辑,请 detach 物化视图: diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index 0aba93e4591..3481e9c54ef 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -1,6 +1,4 @@ - - -# MergeTree +# MergeTree {#table_engines-mergetree} The `MergeTree` engine and other engines of this family (`*MergeTree`) are the most robust ClickHousе table engines. @@ -79,7 +77,7 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa In the example, we set partitioning by month. -We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If, when selecting the data, you define a [SAMPLE](../../query_language/select.md#sample) clause, ClickHouse will return an evenly pseudorandom data sample for a subset of users. +We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If, when selecting the data, you define a [SAMPLE](../../query_language/select.md#select-sample-clause) clause, ClickHouse will return an evenly pseudorandom data sample for a subset of users. `index_granularity` could be omitted because 8192 is the default value. diff --git a/docs/zh/operations/table_engines/replication.md b/docs/zh/operations/table_engines/replication.md index c7b2f97ee16..9e1c7a83ea0 100644 --- a/docs/zh/operations/table_engines/replication.md +++ b/docs/zh/operations/table_engines/replication.md @@ -1,6 +1,4 @@ - - -# Data Replication +# Data Replication {#table_engines-replication} Replication is only supported for tables in the MergeTree family: diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md index 9b618b1eafd..a5da211946f 100644 --- a/docs/zh/operations/table_engines/summingmergetree.md +++ b/docs/zh/operations/table_engines/summingmergetree.md @@ -90,7 +90,7 @@ SELECT key, sum(value) FROM summtt GROUP BY key When data are inserted into a table, they are saved as-is. Clickhouse merges the inserted parts of data periodically and this is when rows with the same primary key are summed and replaced with one for each resulting part of data. -ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) an aggregate function [sum()](../../query_language/agg_functions/reference.md#sum) and `GROUP BY` clause should be used in a query as described in the example above. +ClickHouse can merge the data parts so that different resulting parts of data cat consist rows with the same primary key, i.e. the summation will be incomplete. Therefore (`SELECT`) an aggregate function [sum()](../../query_language/agg_functions/reference.md#agg_function-sum) and `GROUP BY` clause should be used in a query as described in the example above. ### Common rules for summation @@ -104,7 +104,7 @@ The values are not summarized for columns in the primary key. ### The Summation in the AggregateFunction Columns -For columns of [AggregateFunction type](../../data_types/nested_data_structures/aggregatefunction.md#data_type-aggregatefunction) ClickHouse behaves as [AggregatingMergeTree](aggregatingmergetree.md) engine aggregating according to the function. +For columns of [AggregateFunction type](../../data_types/nested_data_structures/aggregatefunction.md) ClickHouse behaves as [AggregatingMergeTree](aggregatingmergetree.md) engine aggregating according to the function. ### Nested Structures diff --git a/docs/zh/operations/table_engines/url.md b/docs/zh/operations/table_engines/url.md index 77e690c4de5..1cf803c763c 100644 --- a/docs/zh/operations/table_engines/url.md +++ b/docs/zh/operations/table_engines/url.md @@ -1,6 +1,4 @@ - - -# URL(URL, Format) +# URL(URL, Format) {#table_engines-url} Manages data on a remote HTTP/HTTPS server. This engine is similar to the [File](file.md) engine. diff --git a/docs/zh/query_language/create.md b/docs/zh/query_language/create.md index 1cc5fe0febc..332b53f812a 100644 --- a/docs/zh/query_language/create.md +++ b/docs/zh/query_language/create.md @@ -43,7 +43,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... 以上所有情况,如果指定了`IF NOT EXISTS`,那么在该表已经存在的情况下,查询不会返回任何错误。在这种情况下,查询几乎不会做任何事情。 -在`ENGINE`子句后还可能存在一些其他的子句,更详细的信息可以参考[表引擎](../operations/table_engines/index.md#table_engines)中关于建表的描述。 +在`ENGINE`子句后还可能存在一些其他的子句,更详细的信息可以参考[表引擎](../operations/table_engines/index.md)中关于建表的描述。 ### 默认值 diff --git a/docs/zh/query_language/select.md b/docs/zh/query_language/select.md index 2da781c39cb..607f26ded71 100644 --- a/docs/zh/query_language/select.md +++ b/docs/zh/query_language/select.md @@ -44,7 +44,7 @@ FROM子句规定了将从哪个表、或子查询、或表函数中读取数据 最后的FINAL修饰符仅能够被使用在SELECT from CollapsingMergeTree场景中。当你为FROM指定了FINAL修饰符时,你的查询结果将会在查询过程中被聚合。需要注意的是,在这种情况下,查询将在单个流中读取所有相关的主键列,同时对需要的数据进行合并。这意味着,当使用FINAL修饰符时,查询将会处理的更慢。在大多数情况下,你应该避免使用FINAL修饰符。更多信息,请参阅“CollapsingMergeTree引擎”部分。 -### SAMPLE 子句 +### SAMPLE 子句 {#select-sample-clause} 通过SAMPLE子句用户可以进行近似查询处理,近似查询处理仅能工作在MergeTree\*类型的表中,并且在创建表时需要您指定采样表达式(参见“MergeTree 引擎”部分)。 @@ -81,7 +81,7 @@ ORDER BY PageViews DESC LIMIT 1000 例如,我们可以使用采样的方式获取到与不进行采样相同的用户ID的列表。这将表明,你可以在IN子查询中使用采样,或者使用采样的结果与其他查询进行关联。 -### ARRAY JOIN 子句 +### ARRAY JOIN 子句 {#select-array-join-clause} ARRAY JOIN子句可以帮助查询进行与数组和nested数据类型的连接。它有点类似arrayJoin函数,但它的功能更广泛。 @@ -339,7 +339,7 @@ ARRAY JOIN nest AS n, arrayEnumerate(`nest.x`) AS num JOIN子句用于连接数据,作用与[SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL))的定义相同。 !!! info "注意" - 与[ARRAY JOIN](#array-join)没有关系. + 与[ARRAY JOIN](#select-array-join-clause)没有关系. ``` sql @@ -373,7 +373,7 @@ FROM 当使用`GLOBAL ... JOIN`,首先会在请求服务器上计算右表并以临时表的方式将其发送到所有服务器。这时每台服务器将直接使用它进行计算。 -使用`GLOBAL`时需要小心。更多信息,参阅[Distributed subqueries](#distributed-subqueries)部分。 +使用`GLOBAL`时需要小心。更多信息,参阅[Distributed subqueries](#select-distributed-subqueries)部分。 **使用建议** @@ -441,7 +441,7 @@ LIMIT 10 JOIN的行为受[join_use_nulls](../operations/settings/settings.md)的影响。当`join_use_nulls=1`时,`JOIN`的工作与SQL标准相同。 -如果JOIN的key是[Nullable](../data_types/nullable.md#data_types-nullable)类型的字段,则其中至少一个存在[NULL](syntax.md)值的key不会被关联。 +如果JOIN的key是[Nullable](../data_types/nullable.md)类型的字段,则其中至少一个存在[NULL](syntax.md)值的key不会被关联。 ### WHERE 子句 @@ -747,7 +747,7 @@ UNION ALL中的查询可以同时运行,它们的结果将被混合到一起 当使用命令行客户端时,数据以内部高效的格式在服务器和客户端之间进行传递。客户端将单独的解析FORMAT子句,以帮助数据格式的转换(这将减轻网络和服务器的负载)。 -### IN 运算符 +### IN 运算符 {#select-in-operators} 对于`IN`、`NOT IN`、`GLOBAL IN`、`GLOBAL NOT IN`操作符被分别实现,因为它们的功能非常丰富。 @@ -846,7 +846,7 @@ FROM t_null ``` -#### 分布式子查询 +#### 分布式子查询 {#select-distributed-subqueries} 对于带有子查询的(类似与JOINs)IN中,有两种选择:普通的`IN`/`JOIN`与`GLOBAL IN` / `GLOBAL JOIN`。它们对于分布式查询的处理运行方式是不同的。 From 7b0fcecdf22519264cdf31e4ae300501023e235e Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 21 Dec 2018 22:40:49 +0300 Subject: [PATCH 145/181] minor docs improvements (#3906) * CLICKHOUSE-4063: less manual html @ index.md * CLICKHOUSE-4063: recommend markdown="1" in README.md * CLICKHOUSE-4003: manually purge custom.css for now * CLICKHOUSE-4064: expand
before any print (including to pdf) * CLICKHOUSE-3927: rearrange interfaces/formats.md a bit * CLICKHOUSE-3306: add few http headers * Remove copy-paste introduced in #3392 * Hopefully better chinese fonts #3392 * get rid of tabs @ custom.css * Apply comments and patch from #3384 * Add jdbc.md to ToC and some translation, though it still looks badly incomplete * minor punctuation * Add some backlinks to official website from mirrors that just blindly take markdown sources * Do not make fonts extra light * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's//g' {} * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's/ sql/g' {} * Remove outdated stuff from roadmap.md * Not so light font on front page too * Refactor Chinese formats.md to match recent changes in other languages * Update some links on front page * Remove some outdated comment * Add twitter link to front page * More front page links tuning * Add Amsterdam meetup link * Smaller font to avoid second line * Add Amsterdam link to README.md * Proper docs nav translation * Back to 300 font-weight except Chinese * fix docs build * Update Amsterdam link * remove symlinks * more zh punctuation * apply lost comment by @zhang2014 * Apply comments by @zhang2014 from #3417 * Remove Beijing link * rm incorrect symlink * restore content of docs/zh/operations/table_engines/index.md * CLICKHOUSE-3751: stem terms while searching docs * CLICKHOUSE-3751: use English stemmer in non-English docs too * CLICKHOUSE-4135 fix * Remove past meetup link * Add blog link to top nav * Add ContentSquare article link * Add form link to front page + refactor some texts * couple markup fixes * minor * Introduce basic ODBC driver page in docs * More verbose 3rd party libs disclaimer * Put third-party stuff into a separate folder * Separate third-party stuff in ToC too * Update links * Move stuff that is not really (only) a client library into a separate page * Add clickhouse-hdfs-loader link * Some introduction for "interfaces" section * Rewrite tcp.md * http_interface.md -> http.md * fix link * Remove unconvenient error for now * try to guess anchor instead of failing * remove symlink * Remove outdated info from introduction * remove ru roadmap.md * replace ru roadmap.md with symlink * Update roadmap.md * lost file * Title case in toc_en.yml * Sync "Functions" ToC section with en * Remove reference to pretty old ClickHouse release from docs * couple lost symlinks in fa * Close quote in proper place * Rewrite en/getting_started/index.md * Sync en<>ru getting_started/index.md * minor changes * Some gui.md refactoring * Translate DataGrip section to ru * Translate DataGrip section to zh * Translate DataGrip section to fa * Translate DBeaver section to fa * Translate DBeaver section to zh * Split third-party GUI to open-source and commercial * Mention some RDBMS integrations + ad-hoc translation fixes * Add rel="external nofollow" to outgoing links from docs * Lost blank lines * Fix class name * More rel="external nofollow" * Apply suggestions by @sundy-li * Mobile version of front page improvements * test * test 2 * test 3 * Update LICENSE * minor docs fix * Highlight current article as suggested by @sundy-li * fix link destination * Introduce backup.md (only "en" for now) * Mention INSERT+SELECT in backup.md * Some improvements for replication.md * Add backup.md to toc * Mention clickhouse-backup tool * Mention LightHouse in third-party GUI list * Introduce interfaces/third-party/proxy.md * Add clickhouse-bulk to proxy.md * Major extension of integrations.md contents * fix link target * remove unneeded file * better toc item name * fix markdown * better ru punctuation * Add yet another possible backup approach * Simplify copying permalinks to headers * Support non-eng link anchors in docs + update some deps * Generate anchors for single-page mode automatically * Remove anchors to top of pages * Remove anchors that nobody links to * build fixes * fix few links * restore css * fix some links * restore gifs * fix lost words * more docs fixes * docs fixes * NULL anchor * update urllib3 dependency * more fixes * Remove excessive content from print version * Try short license again * Back to long license for now * Introduce anchor integrity checks for single-page docs * Add --save-raw-single-page option to build.py (helps to debug incorrect anchors) * fix kafka engine links * fix one class of broken anchors * fix some broken links * Add https://github.com/hatarist/clickhouse-cli to third-party section (in gui.md for now, maybe will add cli.md later) * fix one more class of links to nowhere * less duplicate anchors * get rid of weird anchors * fix anchor * fix link * fix couple links * rearrange integrations.md a bit + sync zh version * Mention nagios plugin in other languages * port summingmergetree.md fix to zh * Make doc links to nowhere fatal * additional check in markdown extension * add option to skip pdf --- docs/en/interfaces/third-party/integrations.md | 6 +++--- docs/fa/interfaces/third-party/integrations.md | 8 +++++--- docs/ru/interfaces/third-party/integrations.md | 8 +++++--- docs/tools/build.py | 12 +++++++----- docs/tools/mdx_clickhouse.py | 7 ++++--- docs/tools/test.py | 1 + docs/zh/interfaces/third-party/integrations.md | 5 +++++ docs/zh/operations/table_engines/summingmergetree.md | 2 +- 8 files changed, 31 insertions(+), 18 deletions(-) diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index daa2a73958f..552886abe80 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -19,6 +19,9 @@ - Object storages - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) +- Configuration management + - [puppet](https://puppet.com) + - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) - Monitoring - [Graphite](https://graphiteapp.org) - [graphouse](https://github.com/yandex/graphouse) @@ -33,9 +36,6 @@ - Logging - [fluentd](https://www.fluentd.org) - [loghouse](https://github.com/flant/loghouse) (for [Kubernetes](https://kubernetes.io)) -- Configuration management - - [puppet](https://puppet.com) - - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) ## Programming Language Ecosystems diff --git a/docs/fa/interfaces/third-party/integrations.md b/docs/fa/interfaces/third-party/integrations.md index e0258730e14..bcb741dc092 100644 --- a/docs/fa/interfaces/third-party/integrations.md +++ b/docs/fa/interfaces/third-party/integrations.md @@ -21,6 +21,9 @@ - فروشگاه شی - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) +- مدیریت تنظیمات + - [puppet](https://puppet.com) + - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) - نظارت بر - [Graphite](https://graphiteapp.org) - [graphouse](https://github.com/yandex/graphouse) @@ -30,12 +33,11 @@ - [Prometheus](https://prometheus.io/) - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) + - [Nagios](https://www.nagios.org/) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) - ثبت نام - [fluentd](https://www.fluentd.org) - [loghouse](https://github.com/flant/loghouse) (برای [Kubernetes](https://kubernetes.io)) -- مدیریت تنظیمات - - [puppet](https://puppet.com) - - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) ## اکوسیستم زبان برنامه نویسی diff --git a/docs/ru/interfaces/third-party/integrations.md b/docs/ru/interfaces/third-party/integrations.md index 0d7b92e26d3..776da38f0ad 100644 --- a/docs/ru/interfaces/third-party/integrations.md +++ b/docs/ru/interfaces/third-party/integrations.md @@ -18,6 +18,9 @@ - Хранилища объектов - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) +- Системы управления конфигурацией + - [puppet](https://puppet.com) + - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) - Мониторинг - [Graphite](https://graphiteapp.org) - [graphouse](https://github.com/yandex/graphouse) @@ -27,12 +30,11 @@ - [Prometheus](https://prometheus.io/) - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) + - [Nagios](https://www.nagios.org/) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) - Логирование - [fluentd](https://www.fluentd.org) - [loghouse](https://github.com/flant/loghouse) (для [Kubernetes](https://kubernetes.io)) -- Системы управления конфигурацией - - [puppet](https://puppet.com) - - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) ## Экосистемы вокруг языков программирования diff --git a/docs/tools/build.py b/docs/tools/build.py index 73676f0f30c..e3c90f2b956 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -178,11 +178,12 @@ def build_single_page_version(lang, args, cfg): single_page_output_path ) - single_page_index_html = os.path.abspath(os.path.join(single_page_output_path, 'index.html')) - single_page_pdf = single_page_index_html.replace('index.html', 'clickhouse_%s.pdf' % lang) - create_pdf_command = ['wkhtmltopdf', '--print-media-type', single_page_index_html, single_page_pdf] - logging.debug(' '.join(create_pdf_command)) - subprocess.check_call(' '.join(create_pdf_command), shell=True) + if not args.skip_pdf: + single_page_index_html = os.path.abspath(os.path.join(single_page_output_path, 'index.html')) + single_page_pdf = single_page_index_html.replace('index.html', 'clickhouse_%s.pdf' % lang) + create_pdf_command = ['wkhtmltopdf', '--print-media-type', single_page_index_html, single_page_pdf] + logging.debug(' '.join(create_pdf_command)) + subprocess.check_call(' '.join(create_pdf_command), shell=True) with temp_dir() as test_dir: cfg.load_dict({ @@ -229,6 +230,7 @@ if __name__ == '__main__': arg_parser.add_argument('--theme-dir', default='mkdocs-material-theme') arg_parser.add_argument('--output-dir', default='build') arg_parser.add_argument('--skip-single-page', action='store_true') + arg_parser.add_argument('--skip-pdf', action='store_true') arg_parser.add_argument('--save-raw-single-page', type=str) arg_parser.add_argument('--verbose', action='store_true') diff --git a/docs/tools/mdx_clickhouse.py b/docs/tools/mdx_clickhouse.py index 8f72d4eec2f..ae57d1309e6 100755 --- a/docs/tools/mdx_clickhouse.py +++ b/docs/tools/mdx_clickhouse.py @@ -43,9 +43,10 @@ class ClickHouseLinkPattern(ClickHouseLinkMixin, markdown.inlinepatterns.LinkPat class ClickHousePreprocessor(markdown.util.Processor): def run(self, lines): - for line in lines: - if '' not in line: - yield line + if os.getenv('QLOUD_TOKEN'): + for line in lines: + if '' not in line: + yield line class ClickHouseMarkdown(markdown.extensions.Extension): diff --git a/docs/tools/test.py b/docs/tools/test.py index 366f3532e88..48e246507f4 100755 --- a/docs/tools/test.py +++ b/docs/tools/test.py @@ -34,6 +34,7 @@ def test_single_page(input_path, lang): logging.error('Found %d duplicate anchor points' % duplicate_anchor_points) if links_to_nowhere: logging.error('Found %d links to nowhere' % links_to_nowhere) + sys.exit(10) assert len(anchor_points) > 10, 'Html parsing is probably broken' diff --git a/docs/zh/interfaces/third-party/integrations.md b/docs/zh/interfaces/third-party/integrations.md index fb5467f3192..46ad1b690c8 100644 --- a/docs/zh/interfaces/third-party/integrations.md +++ b/docs/zh/interfaces/third-party/integrations.md @@ -18,6 +18,9 @@ - 对象存储 - [S3](https://en.wikipedia.org/wiki/Amazon_S3) - [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup) +- 配置管理 + - [puppet](https://puppet.com) + - [innogames/clickhouse](https://forge.puppet.com/innogames/clickhouse) - 监控 - [Graphite](https://graphiteapp.org) - [graphouse](https://github.com/yandex/graphouse) @@ -27,6 +30,8 @@ - [Prometheus](https://prometheus.io/) - [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter) - [PromHouse](https://github.com/Percona-Lab/PromHouse) + - [Nagios](https://www.nagios.org/) + - [check_clickhouse](https://github.com/exogroup/check_clickhouse/) - 记录 - [fluentd](https://www.fluentd.org) - [loghouse](https://github.com/flant/loghouse) (对于 [Kubernetes](https://kubernetes.io)) diff --git a/docs/zh/operations/table_engines/summingmergetree.md b/docs/zh/operations/table_engines/summingmergetree.md index a5da211946f..2013c7142b9 100644 --- a/docs/zh/operations/table_engines/summingmergetree.md +++ b/docs/zh/operations/table_engines/summingmergetree.md @@ -13,7 +13,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) ENGINE = MergeTree() +) ENGINE = SummingMergeTree() [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] From 94948cb58721d3224a2b6eba3aade49cdb4a4110 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Dec 2018 18:40:51 +0300 Subject: [PATCH 146/181] Less garbage [#CLICKHOUSE-2] --- dbms/src/Functions/arrayEnumerateExtended.h | 198 +++++--------------- 1 file changed, 52 insertions(+), 146 deletions(-) diff --git a/dbms/src/Functions/arrayEnumerateExtended.h b/dbms/src/Functions/arrayEnumerateExtended.h index 159d607d09c..fa45119577f 100644 --- a/dbms/src/Functions/arrayEnumerateExtended.h +++ b/dbms/src/Functions/arrayEnumerateExtended.h @@ -61,21 +61,10 @@ private: static constexpr size_t INITIAL_SIZE_DEGREE = 9; template - bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool execute128bit( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns); - - void executeHashed( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values); + bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); + bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); + bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); + bool executeHashed(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); }; @@ -86,9 +75,8 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C ColumnRawPtrs data_columns; data_columns.reserve(arguments.size()); - bool has_nullable_columns = false; - Columns array_holders; + ColumnPtr offsets_holder; for (size_t i = 0; i < arguments.size(); ++i) { const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column; @@ -105,6 +93,7 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C array = checkAndGetColumn(array_holders.back().get()); } + offsets_holder = array->getOffsetsPtr(); const ColumnArray::Offsets & offsets_i = array->getOffsets(); if (i == 0) offsets = &offsets_i; @@ -117,26 +106,22 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C } size_t num_columns = data_columns.size(); - ColumnRawPtrs original_data_columns(num_columns); - ColumnRawPtrs null_maps(num_columns); + const NullMap * null_map = nullptr; for (size_t i = 0; i < num_columns; ++i) { - original_data_columns[i] = data_columns[i]; - if (data_columns[i]->isColumnNullable()) { - has_nullable_columns = true; const auto & nullable_col = static_cast(*data_columns[i]); - data_columns[i] = &nullable_col.getNestedColumn(); - null_maps[i] = &nullable_col.getNullMapColumn(); + + if (num_columns == 1) + data_columns[i] = &nullable_col.getNestedColumn(); + + null_map = &nullable_col.getNullMapData(); + break; } - else - null_maps[i] = nullptr; } - const ColumnArray * first_array = checkAndGetColumn(block.getByPosition(arguments.at(0)).column.get()); - const IColumn * first_null_map = null_maps[0]; auto res_nested = ColumnUInt32::create(); ColumnUInt32::Container & res_values = res_nested->getData(); @@ -145,57 +130,42 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C if (num_columns == 1) { - if (!(executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeString (first_array, first_null_map, res_values))) - executeHashed(*offsets, original_data_columns, res_values); + executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeString(*offsets, *data_columns[0], null_map, res_values) + || executeHashed(*offsets, data_columns, res_values); } else { - if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns)) - executeHashed(*offsets, original_data_columns, res_values); + execute128bit(*offsets, data_columns, res_values) + || executeHashed(*offsets, data_columns, res_values); } - block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), first_array->getOffsetsPtr()); + block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_holder); } template template -bool FunctionArrayEnumerateExtended::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +bool FunctionArrayEnumerateExtended::executeNumber( + const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnVector * nested = checkAndGetColumn>(inner_col); - if (!nested) + const ColumnVector * data_concrete = typeid_cast *>(&data); + if (!data_concrete) return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); - const typename ColumnVector::Container & values = nested->getData(); + const auto & values = data_concrete->getData(); using ValuesToIndices = ClearableHashMap, HashTableGrower, HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - ValuesToIndices indices; size_t prev_off = 0; if constexpr (std::is_same_v) @@ -208,7 +178,7 @@ bool FunctionArrayEnumerateExtended::executeNumber(const ColumnArray * size_t off = offsets[i]; for (size_t j = prev_off; j < off; ++j) { - if (null_map_data && ((*null_map_data)[j] == 1)) + if (null_map && (*null_map)[j]) res_values[j] = ++null_count; else res_values[j] = ++indices[values[j]]; @@ -227,7 +197,7 @@ bool FunctionArrayEnumerateExtended::executeNumber(const ColumnArray * size_t off = offsets[i]; for (size_t j = prev_off; j < off; ++j) { - if (null_map_data && ((*null_map_data)[j] == 1)) + if (null_map && (*null_map)[j]) { if (!null_index) null_index = ++rank; @@ -248,32 +218,17 @@ bool FunctionArrayEnumerateExtended::executeNumber(const ColumnArray * } template -bool FunctionArrayEnumerateExtended::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +bool FunctionArrayEnumerateExtended::executeString( + const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnString * nested = checkAndGetColumn(inner_col); - if (!nested) + const ColumnString * values = typeid_cast(&data); + if (!values) return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); size_t prev_off = 0; using ValuesToIndices = ClearableHashMap, HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - ValuesToIndices indices; if constexpr (std::is_same_v) { @@ -285,10 +240,10 @@ bool FunctionArrayEnumerateExtended::executeString(const ColumnArray * size_t off = offsets[i]; for (size_t j = prev_off; j < off; ++j) { - if (null_map_data && ((*null_map_data)[j] == 1)) + if (null_map && (*null_map)[j]) res_values[j] = ++null_count; else - res_values[j] = ++indices[nested->getDataAt(j)]; + res_values[j] = ++indices[values->getDataAt(j)]; } prev_off = off; } @@ -304,7 +259,7 @@ bool FunctionArrayEnumerateExtended::executeString(const ColumnArray * size_t off = offsets[i]; for (size_t j = prev_off; j < off; ++j) { - if (null_map_data && ((*null_map_data)[j] == 1)) + if (null_map && (*null_map)[j]) { if (!null_index) null_index = ++rank; @@ -312,7 +267,7 @@ bool FunctionArrayEnumerateExtended::executeString(const ColumnArray * } else { - auto & idx = indices[nested->getDataAt(j)]; + auto & idx = indices[values->getDataAt(j)]; if (!idx) idx = ++rank; res_values[j] = idx; @@ -328,9 +283,7 @@ template bool FunctionArrayEnumerateExtended::execute128bit( const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns) + ColumnUInt32::Container & res_values) { size_t count = columns.size(); size_t keys_bytes = 0; @@ -343,8 +296,6 @@ bool FunctionArrayEnumerateExtended::execute128bit( key_sizes[j] = columns[j]->sizeOfValueIfFixed(); keys_bytes += key_sizes[j]; } - if (has_nullable_columns) - keys_bytes += std::tuple_size>::value; if (keys_bytes > 16) return false; @@ -362,29 +313,7 @@ bool FunctionArrayEnumerateExtended::execute128bit( indices.clear(); size_t off = offsets[i]; for (size_t j = prev_off; j < off; ++j) - { - if (has_nullable_columns) - { - KeysNullMap bitmap{}; - - for (size_t i = 0; i < columns.size(); ++i) - { - if (null_maps[i]) - { - const auto & null_map = static_cast(*null_maps[i]).getData(); - if (null_map[j] == 1) - { - size_t bucket = i / 8; - size_t offset = i % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - res_values[j] = ++indices[packFixed(j, count, columns, key_sizes, bitmap)]; - } - else - res_values[j] = ++indices[packFixed(j, count, columns, key_sizes)]; - } + res_values[j] = ++indices[packFixed(j, count, columns, key_sizes)]; prev_off = off; } } @@ -398,35 +327,10 @@ bool FunctionArrayEnumerateExtended::execute128bit( size_t rank = 0; for (size_t j = prev_off; j < off; ++j) { - if (has_nullable_columns) - { - KeysNullMap bitmap{}; - - for (size_t i = 0; i < columns.size(); ++i) - { - if (null_maps[i]) - { - const auto & null_map = static_cast(*null_maps[i]).getData(); - if (null_map[j] == 1) - { - size_t bucket = i / 8; - size_t offset = i % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - auto &idx = indices[packFixed(j, count, columns, key_sizes, bitmap)]; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } - else - { - auto &idx = indices[packFixed(j, count, columns, key_sizes)];; - if (!idx) - idx = ++rank; - res_values[j] = idx; - } + auto &idx = indices[packFixed(j, count, columns, key_sizes)];; + if (!idx) + idx = ++rank; + res_values[j] = idx; } prev_off = off; } @@ -436,7 +340,7 @@ bool FunctionArrayEnumerateExtended::execute128bit( } template -void FunctionArrayEnumerateExtended::executeHashed( +bool FunctionArrayEnumerateExtended::executeHashed( const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values) @@ -480,6 +384,8 @@ void FunctionArrayEnumerateExtended::executeHashed( prev_off = off; } } + + return true; } } From fac239147d2f856531a8246510b6d54e5ba2099c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Dec 2018 19:19:16 +0300 Subject: [PATCH 147/181] Added test #3909 --- .../0_stateless/00808_array_enumerate_segfault.reference | 2 ++ .../queries/0_stateless/00808_array_enumerate_segfault.sql | 3 +++ dbms/tests/queries/bugs/fuzzy.sql | 2 -- 3 files changed, 5 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.reference create mode 100644 dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql diff --git a/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.reference b/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.reference new file mode 100644 index 00000000000..ad1021a5788 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.reference @@ -0,0 +1,2 @@ +[] +[1] diff --git a/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql b/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql new file mode 100644 index 00000000000..e1a0964932d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql @@ -0,0 +1,3 @@ +SELECT arrayEnumerateUniq(anyHeavy([]), []); +SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); -- { serverError 190 } +SELECT arrayEnumerateDense([STDDEV_SAMP(NULL, 910947.571364)], [NULL]); diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index f81140ba8c9..9a5fd36fbb3 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -3,9 +3,7 @@ SELECT globalNotIn(['"wh'], [NULL]); SELECT globalIn([''], [NULL]) SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); SELECT truncate(895, -16); -SELECT arrayEnumerateUniq(anyHeavy([]), []); SELECT notIn([['']], [[NULL]]); SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; SELECT addDays((CAST((96.338) AS DateTime)), -3); -SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); From 83cf88c9c1222ab0701b814c70dffd9334ac4de8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 04:41:03 +0300 Subject: [PATCH 148/181] Less garbage #3909 --- dbms/src/Functions/arrayEnumerateExtended.h | 27 +-- dbms/src/Functions/arrayUniq.cpp | 196 ++++++-------------- 2 files changed, 71 insertions(+), 152 deletions(-) diff --git a/dbms/src/Functions/arrayEnumerateExtended.h b/dbms/src/Functions/arrayEnumerateExtended.h index fa45119577f..a0ceed86b4a 100644 --- a/dbms/src/Functions/arrayEnumerateExtended.h +++ b/dbms/src/Functions/arrayEnumerateExtended.h @@ -72,12 +72,12 @@ template void FunctionArrayEnumerateExtended::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) { const ColumnArray::Offsets * offsets = nullptr; - ColumnRawPtrs data_columns; - data_columns.reserve(arguments.size()); + size_t num_arguments = arguments.size(); + ColumnRawPtrs data_columns(num_arguments); Columns array_holders; - ColumnPtr offsets_holder; - for (size_t i = 0; i < arguments.size(); ++i) + ColumnPtr offsets_column; + for (size_t i = 0; i < num_arguments; ++i) { const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column; const ColumnArray * array = checkAndGetColumn(array_ptr.get()); @@ -93,28 +93,29 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C array = checkAndGetColumn(array_holders.back().get()); } - offsets_holder = array->getOffsetsPtr(); const ColumnArray::Offsets & offsets_i = array->getOffsets(); if (i == 0) + { offsets = &offsets_i; + offsets_column = array->getOffsetsPtr(); + } else if (offsets_i != *offsets) throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); auto * array_data = &array->getData(); - data_columns.push_back(array_data); + data_columns[i] = array_data; } - size_t num_columns = data_columns.size(); const NullMap * null_map = nullptr; - for (size_t i = 0; i < num_columns; ++i) + for (size_t i = 0; i < num_arguments; ++i) { if (data_columns[i]->isColumnNullable()) { const auto & nullable_col = static_cast(*data_columns[i]); - if (num_columns == 1) + if (num_arguments == 1) data_columns[i] = &nullable_col.getNestedColumn(); null_map = &nullable_col.getNullMapData(); @@ -128,7 +129,7 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C if (!offsets->empty()) res_values.resize(offsets->back()); - if (num_columns == 1) + if (num_arguments == 1) { executeNumber(*offsets, *data_columns[0], null_map, res_values) || executeNumber(*offsets, *data_columns[0], null_map, res_values) @@ -149,7 +150,7 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C || executeHashed(*offsets, data_columns, res_values); } - block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_holder); + block.getByPosition(result).column = ColumnArray::create(std::move(res_nested), offsets_column); } @@ -158,7 +159,7 @@ template bool FunctionArrayEnumerateExtended::executeNumber( const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const ColumnVector * data_concrete = typeid_cast *>(&data); + const ColumnVector * data_concrete = checkAndGetColumn>(&data); if (!data_concrete) return false; const auto & values = data_concrete->getData(); @@ -221,7 +222,7 @@ template bool FunctionArrayEnumerateExtended::executeString( const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const ColumnString * values = typeid_cast(&data); + const ColumnString * values = checkAndGetColumn(&data); if (!values) return false; diff --git a/dbms/src/Functions/arrayUniq.cpp b/dbms/src/Functions/arrayUniq.cpp index 3a90d76aae6..93af86a3441 100644 --- a/dbms/src/Functions/arrayUniq.cpp +++ b/dbms/src/Functions/arrayUniq.cpp @@ -63,37 +63,23 @@ private: static constexpr size_t INITIAL_SIZE_DEGREE = 9; template - bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - - bool execute128bit( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns); - - void executeHashed( - const ColumnArray::Offsets & offsets, - const ColumnRawPtrs & columns, - ColumnUInt32::Container & res_values); + bool executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); + bool executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values); + bool execute128bit(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); + bool executeHashed(const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values); }; void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) { - Columns array_columns(arguments.size()); const ColumnArray::Offsets * offsets = nullptr; - ColumnRawPtrs data_columns(arguments.size()); - ColumnRawPtrs original_data_columns(arguments.size()); - ColumnRawPtrs null_maps(arguments.size()); + size_t num_arguments = arguments.size(); + ColumnRawPtrs data_columns(num_arguments); - bool has_nullable_columns = false; - - for (size_t i = 0; i < arguments.size(); ++i) + Columns array_holders; + for (size_t i = 0; i < num_arguments; ++i) { - ColumnPtr array_ptr = block.getByPosition(arguments[i]).column; + const ColumnPtr & array_ptr = block.getByPosition(arguments[i]).column; const ColumnArray * array = checkAndGetColumn(array_ptr.get()); if (!array) { @@ -101,14 +87,12 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen block.getByPosition(arguments[i]).column.get()); if (!const_array) throw Exception("Illegal column " + block.getByPosition(arguments[i]).column->getName() - + " of " + toString(i + 1) + getOrdinalSuffix(i + 1) + " argument of function " + getName(), + + " of " + toString(i + 1) + "-th argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); - array_ptr = const_array->convertToFullColumn(); - array = static_cast(array_ptr.get()); + array_holders.emplace_back(const_array->convertToFullColumn()); + array = checkAndGetColumn(array_holders.back().get()); } - array_columns[i] = array_ptr; - const ColumnArray::Offsets & offsets_i = array->getOffsets(); if (i == 0) offsets = &offsets_i; @@ -116,78 +100,65 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen throw Exception("Lengths of all arrays passed to " + getName() + " must be equal.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); - data_columns[i] = &array->getData(); - original_data_columns[i] = data_columns[i]; - - if (data_columns[i]->isColumnNullable()) - { - has_nullable_columns = true; - const auto & nullable_col = static_cast(*data_columns[i]); - data_columns[i] = &nullable_col.getNestedColumn(); - null_maps[i] = &nullable_col.getNullMapColumn(); - } - else - null_maps[i] = nullptr; + auto * array_data = &array->getData(); + data_columns[i] = array_data; } - const ColumnArray * first_array = static_cast(array_columns[0].get()); - const IColumn * first_null_map = null_maps[0]; - auto res = ColumnUInt32::create(); + const NullMap * null_map = nullptr; + for (size_t i = 0; i < num_arguments; ++i) + { + if (data_columns[i]->isColumnNullable()) + { + const auto & nullable_col = static_cast(*data_columns[i]); + + if (num_arguments == 1) + data_columns[i] = &nullable_col.getNestedColumn(); + + null_map = &nullable_col.getNullMapData(); + break; + } + } + + auto res = ColumnUInt32::create(); ColumnUInt32::Container & res_values = res->getData(); res_values.resize(offsets->size()); - if (arguments.size() == 1) + if (num_arguments == 1) { - if (!(executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeNumber(first_array, first_null_map, res_values) - || executeString(first_array, first_null_map, res_values))) - executeHashed(*offsets, original_data_columns, res_values); + executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeNumber(*offsets, *data_columns[0], null_map, res_values) + || executeString(*offsets, *data_columns[0], null_map, res_values) + || executeHashed(*offsets, data_columns, res_values); } else { - if (!execute128bit(*offsets, data_columns, null_maps, res_values, has_nullable_columns)) - executeHashed(*offsets, original_data_columns, res_values); + execute128bit(*offsets, data_columns, res_values) + || executeHashed(*offsets, data_columns, res_values); } block.getByPosition(result).column = std::move(res); } template -bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +bool FunctionArrayUniq::executeNumber(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnVector * nested = checkAndGetColumn>(inner_col); + const ColumnVector * nested = checkAndGetColumn>(&data); if (!nested) return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); - const typename ColumnVector::Container & values = nested->getData(); + const auto & values = nested->getData(); using Set = ClearableHashSet, HashTableGrower, HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - Set set; ColumnArray::Offset prev_off = 0; for (size_t i = 0; i < offsets.size(); ++i) @@ -197,7 +168,7 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * ColumnArray::Offset off = offsets[i]; for (ColumnArray::Offset j = prev_off; j < off; ++j) { - if (null_map_data && ((*null_map_data)[j] == 1)) + if (null_map && (*null_map)[j]) found_null = true; else set.insert(values[j]); @@ -209,31 +180,15 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn * return true; } -bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values) +bool FunctionArrayUniq::executeString(const ColumnArray::Offsets & offsets, const IColumn & data, const NullMap * null_map, ColumnUInt32::Container & res_values) { - const IColumn * inner_col; - - const auto & array_data = array->getData(); - if (array_data.isColumnNullable()) - { - const auto & nullable_col = static_cast(array_data); - inner_col = &nullable_col.getNestedColumn(); - } - else - inner_col = &array_data; - - const ColumnString * nested = checkAndGetColumn(inner_col); + const ColumnString * nested = checkAndGetColumn(&data); if (!nested) return false; - const ColumnArray::Offsets & offsets = array->getOffsets(); using Set = ClearableHashSet, HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(StringRef)>>; - const PaddedPODArray * null_map_data = nullptr; - if (null_map) - null_map_data = &static_cast(null_map)->getData(); - Set set; ColumnArray::Offset prev_off = 0; for (size_t i = 0; i < offsets.size(); ++i) @@ -243,7 +198,7 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * ColumnArray::Offset off = offsets[i]; for (ColumnArray::Offset j = prev_off; j < off; ++j) { - if (null_map_data && ((*null_map_data)[j] == 1)) + if (null_map && (*null_map)[j]) found_null = true; else set.insert(nested->getDataAt(j)); @@ -259,9 +214,7 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn * bool FunctionArrayUniq::execute128bit( const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, - const ColumnRawPtrs & null_maps, - ColumnUInt32::Container & res_values, - bool has_nullable_columns) + ColumnUInt32::Container & res_values) { size_t count = columns.size(); size_t keys_bytes = 0; @@ -274,8 +227,6 @@ bool FunctionArrayUniq::execute128bit( key_sizes[j] = columns[j]->sizeOfValueIfFixed(); keys_bytes += key_sizes[j]; } - if (has_nullable_columns) - keys_bytes += std::tuple_size>::value; if (keys_bytes > 16) return false; @@ -283,19 +234,6 @@ bool FunctionArrayUniq::execute128bit( using Set = ClearableHashSet, HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; - /// Suppose that, for a given row, each of the N columns has an array whose length is M. - /// Denote arr_i each of these arrays (1 <= i <= N). Then the following is performed: - /// - /// col1 ... colN - /// - /// arr_1[1], ..., arr_N[1] -> pack into a binary blob b1 - /// . - /// . - /// . - /// arr_1[M], ..., arr_N[M] -> pack into a binary blob bM - /// - /// Each binary blob is inserted into a hash table. - /// Set set; ColumnArray::Offset prev_off = 0; for (ColumnArray::Offset i = 0; i < offsets.size(); ++i) @@ -303,29 +241,7 @@ bool FunctionArrayUniq::execute128bit( set.clear(); ColumnArray::Offset off = offsets[i]; for (ColumnArray::Offset j = prev_off; j < off; ++j) - { - if (has_nullable_columns) - { - KeysNullMap bitmap{}; - - for (ColumnArray::Offset i = 0; i < columns.size(); ++i) - { - if (null_maps[i]) - { - const auto & null_map = static_cast(*null_maps[i]).getData(); - if (null_map[j] == 1) - { - ColumnArray::Offset bucket = i / 8; - ColumnArray::Offset offset = i % 8; - bitmap[bucket] |= UInt8(1) << offset; - } - } - } - set.insert(packFixed(j, count, columns, key_sizes, bitmap)); - } - else - set.insert(packFixed(j, count, columns, key_sizes)); - } + set.insert(packFixed(j, count, columns, key_sizes)); res_values[i] = set.size(); prev_off = off; @@ -334,7 +250,7 @@ bool FunctionArrayUniq::execute128bit( return true; } -void FunctionArrayUniq::executeHashed( +bool FunctionArrayUniq::executeHashed( const ColumnArray::Offsets & offsets, const ColumnRawPtrs & columns, ColumnUInt32::Container & res_values) @@ -356,6 +272,8 @@ void FunctionArrayUniq::executeHashed( res_values[i] = set.size(); prev_off = off; } + + return true; } From 19d57c78b58c8de5361789b404dd4bc4039fea9c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 04:46:30 +0300 Subject: [PATCH 149/181] Fixed test #3909 --- dbms/src/Columns/ColumnArray.cpp | 7 +++++++ dbms/src/Columns/ColumnArray.h | 1 + 2 files changed, 8 insertions(+) diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index a3aa421d1c5..eb497ea8f31 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -320,6 +320,13 @@ bool ColumnArray::hasEqualOffsets(const ColumnArray & other) const } +ColumnPtr ColumnArray::convertToFullColumnIfConst() const +{ + /// It is possible to have an array with constant data and non-constant offsets. + /// Example is the result of expression: replicate('hello', [1]) + return ColumnArray::create(data->convertToFullColumnIfConst(), offsets); +} + void ColumnArray::getExtremes(Field & min, Field & max) const { min = Array(); diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index c73cc8faa1e..c2c17c17ed7 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -79,6 +79,7 @@ public: size_t byteSize() const override; size_t allocatedBytes() const override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; + ColumnPtr convertToFullColumnIfConst() const override; void getExtremes(Field & min, Field & max) const override; bool hasEqualOffsets(const ColumnArray & other) const; From 4468462ca64c67479f1cd1a7e2ecae82eab5dda2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 04:48:09 +0300 Subject: [PATCH 150/181] Fixed test #3909 --- .../tests/queries/0_stateless/00808_array_enumerate_segfault.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql b/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql index e1a0964932d..b492d3114f8 100644 --- a/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql +++ b/dbms/tests/queries/0_stateless/00808_array_enumerate_segfault.sql @@ -1,3 +1,4 @@ +SET send_logs_level = 'none'; SELECT arrayEnumerateUniq(anyHeavy([]), []); SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); -- { serverError 190 } SELECT arrayEnumerateDense([STDDEV_SAMP(NULL, 910947.571364)], [NULL]); From dd7325480fccce6445a7bfb9f2cc9461bba3df7b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 05:11:56 +0300 Subject: [PATCH 151/181] Fixed test [#CLICKHOUSE-2] --- dbms/src/Functions/regexpQuoteMeta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/regexpQuoteMeta.cpp b/dbms/src/Functions/regexpQuoteMeta.cpp index cc8f1791578..0ae0a0da97d 100644 --- a/dbms/src/Functions/regexpQuoteMeta.cpp +++ b/dbms/src/Functions/regexpQuoteMeta.cpp @@ -81,7 +81,7 @@ public: while (true) { - const char * next_src_pos = find_first_symbols<'\0', '\\', '|', '(', ')', '^', '$', '.', '[', '?', '*', '+', '{', ':', '-'>(src_pos, src_end); + const char * next_src_pos = find_first_symbols<'\0', '\\', '|', '(', ')', '^', '$', '.', '[', ']', '?', '*', '+', '{', ':', '-'>(src_pos, src_end); size_t bytes_to_copy = next_src_pos - src_pos; size_t old_dst_size = dst_data.size(); From 34bed6c0785d11bdd21af3147c133022382bdd09 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 05:11:56 +0300 Subject: [PATCH 152/181] Fixed test [#CLICKHOUSE-2] --- dbms/src/Functions/regexpQuoteMeta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/regexpQuoteMeta.cpp b/dbms/src/Functions/regexpQuoteMeta.cpp index cc8f1791578..0ae0a0da97d 100644 --- a/dbms/src/Functions/regexpQuoteMeta.cpp +++ b/dbms/src/Functions/regexpQuoteMeta.cpp @@ -81,7 +81,7 @@ public: while (true) { - const char * next_src_pos = find_first_symbols<'\0', '\\', '|', '(', ')', '^', '$', '.', '[', '?', '*', '+', '{', ':', '-'>(src_pos, src_end); + const char * next_src_pos = find_first_symbols<'\0', '\\', '|', '(', ')', '^', '$', '.', '[', ']', '?', '*', '+', '{', ':', '-'>(src_pos, src_end); size_t bytes_to_copy = next_src_pos - src_pos; size_t old_dst_size = dst_data.size(); From 71fad54eb26a14f4f00c8fd3219e69e41f900ed4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 23 Dec 2018 16:21:55 +0800 Subject: [PATCH 153/181] Fix UB. --- dbms/src/Storages/StorageMerge.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 2a2471767e5..e7acd420dec 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -236,7 +236,7 @@ BlockInputStreams StorageMerge::read( else { source_streams.emplace_back(std::make_shared( - header, [=, &real_column_names, &modified_context]() -> BlockInputStreamPtr + header, [=]() mutable -> BlockInputStreamPtr { BlockInputStreams streams = createSourceStreams(query_info, processed_stage, max_block_size, header, storage, struct_lock, real_column_names, From bebaf9d861488b3b5d5c12245e335f20bc7cfd4e Mon Sep 17 00:00:00 2001 From: proller Date: Sun, 23 Dec 2018 17:19:11 +0300 Subject: [PATCH 154/181] Fix includes, Faster compile (#3898) * Fix includes * Faster compile * WTFix * Limit compile and linking jobs according to available memory * Add comment * fix * Remove ALL from copy-headers target * Freebsd fix * Better * cmake: split use libcxx --- CMakeLists.txt | 58 ++----------------- cmake/limit_jobs.cmake | 35 +++++++++++ cmake/use_libcxx.cmake | 49 ++++++++++++++++ dbms/programs/clang/CMakeLists.txt | 4 +- dbms/src/Functions/CMakeLists.txt | 3 +- dbms/src/Functions/FunctionBinaryArithmetic.h | 9 +-- dbms/src/Functions/trim.cpp | 8 ++- utils/check-style/check-include-stat | 8 ++- 8 files changed, 107 insertions(+), 67 deletions(-) create mode 100644 cmake/limit_jobs.cmake create mode 100644 cmake/use_libcxx.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 11bc21e62ed..35e625b60da 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -25,17 +25,10 @@ endif () # Write compile_commands.json set(CMAKE_EXPORT_COMPILE_COMMANDS 1) -set(PARALLEL_COMPILE_JOBS "" CACHE STRING "Define the maximum number of concurrent compilation jobs") -if (PARALLEL_COMPILE_JOBS) - set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool="${PARALLEL_COMPILE_JOBS}") - set(CMAKE_JOB_POOL_COMPILE compile_job_pool) -endif () -set(PARALLEL_LINK_JOBS "" CACHE STRING "Define the maximum number of concurrent link jobs") -if (LLVM_PARALLEL_LINK_JOBS) - set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS}) - set(CMAKE_JOB_POOL_LINK link_job_pool) -endif () +set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") +set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") +include (cmake/limit_jobs.cmake) include (cmake/find_ccache.cmake) @@ -162,51 +155,8 @@ set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fn set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}") -set(THREADS_PREFER_PTHREAD_FLAG ON) -find_package (Threads) -include (cmake/test_compiler.cmake) - -if (OS_LINUX AND COMPILER_CLANG) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}") - - option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" ${HAVE_LIBCXX}) - set (LIBCXX_PATH "" CACHE STRING "Use custom path for libc++. It should be used for MSan.") - - if (USE_LIBCXX) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") # Ok for clang6, for older can cause 'not used option' warning - set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. - if (MAKE_STATIC_LIBRARIES) - execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE) - link_libraries (-nodefaultlibs -Wl,-Bstatic -stdlib=libc++ c++ c++abi gcc_eh ${BUILTINS_LIB_PATH} rt -Wl,-Bdynamic dl pthread m c) - else () - link_libraries (-stdlib=libc++ c++ c++abi) - endif () - - if (LIBCXX_PATH) -# include_directories (SYSTEM BEFORE "${LIBCXX_PATH}/include" "${LIBCXX_PATH}/include/c++/v1") - link_directories ("${LIBCXX_PATH}/lib") - endif () - endif () -endif () - -if (USE_LIBCXX) - set (STATIC_STDLIB_FLAGS "") -else () - set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++") -endif () - -if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") - - # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") -endif () - -if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}") -endif () +include (cmake/use_libcxx.cmake) if (NOT MAKE_STATIC_LIBRARIES) set(CMAKE_POSITION_INDEPENDENT_CODE ON) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake new file mode 100644 index 00000000000..d71c5260240 --- /dev/null +++ b/cmake/limit_jobs.cmake @@ -0,0 +1,35 @@ +# Usage: +# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # In megabytes +# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") +# include (cmake/limit_jobs.cmake) + +cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd + +option(PARALLEL_COMPILE_JOBS "Define the maximum number of concurrent compilation jobs" "") +if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY) + math(EXPR PARALLEL_COMPILE_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/2500) # ~2.5gb max per one compiler + if (NOT PARALLEL_COMPILE_JOBS) + set (PARALLEL_COMPILE_JOBS 1) + endif () +endif () +if (PARALLEL_COMPILE_JOBS) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool=${PARALLEL_COMPILE_JOBS}) + set(CMAKE_JOB_POOL_COMPILE compile_job_pool) +endif () + +option(PARALLEL_LINK_JOBS "Define the maximum number of concurrent link jobs" "") +if (NOT PARALLEL_LINK_JOBS AND AVAILABLE_PHYSICAL_MEMORY) + math(EXPR PARALLEL_LINK_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/4000) # ~4gb max per one linker + if (NOT PARALLEL_LINK_JOBS) + set (PARALLEL_LINK_JOBS 1) + endif () +endif () +if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) + message(STATUS "Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}") +endif () + +if (LLVM_PARALLEL_LINK_JOBS) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS}) + set(CMAKE_JOB_POOL_LINK link_job_pool) +endif () + diff --git a/cmake/use_libcxx.cmake b/cmake/use_libcxx.cmake new file mode 100644 index 00000000000..72aff8f29be --- /dev/null +++ b/cmake/use_libcxx.cmake @@ -0,0 +1,49 @@ +# Uses MAKE_STATIC_LIBRARIES + + +set(THREADS_PREFER_PTHREAD_FLAG ON) +find_package (Threads) + +include (cmake/test_compiler.cmake) +include (cmake/arch.cmake) + +if (OS_LINUX AND COMPILER_CLANG) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}") + + option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" ${HAVE_LIBCXX}) + set (LIBCXX_PATH "" CACHE STRING "Use custom path for libc++. It should be used for MSan.") + + if (USE_LIBCXX) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") # Ok for clang6, for older can cause 'not used option' warning + set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build. + if (MAKE_STATIC_LIBRARIES) + execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-file-name=libclang_rt.builtins-${CMAKE_SYSTEM_PROCESSOR}.a OUTPUT_VARIABLE BUILTINS_LIB_PATH OUTPUT_STRIP_TRAILING_WHITESPACE) + link_libraries (-nodefaultlibs -Wl,-Bstatic -stdlib=libc++ c++ c++abi gcc_eh ${BUILTINS_LIB_PATH} rt -Wl,-Bdynamic dl pthread m c) + else () + link_libraries (-stdlib=libc++ c++ c++abi) + endif () + + if (LIBCXX_PATH) +# include_directories (SYSTEM BEFORE "${LIBCXX_PATH}/include" "${LIBCXX_PATH}/include/c++/v1") + link_directories ("${LIBCXX_PATH}/lib") + endif () + endif () +endif () + +if (USE_LIBCXX) + set (STATIC_STDLIB_FLAGS "") +else () + set (STATIC_STDLIB_FLAGS "-static-libgcc -static-libstdc++") +endif () + +if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (COMPILER_CLANG AND OS_FREEBSD)) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") + + # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${STATIC_STDLIB_FLAGS}") +endif () + +if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAG_NO_PIE}") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}") +endif () diff --git a/dbms/programs/clang/CMakeLists.txt b/dbms/programs/clang/CMakeLists.txt index dec21ac611e..78bfa6b55e7 100644 --- a/dbms/programs/clang/CMakeLists.txt +++ b/dbms/programs/clang/CMakeLists.txt @@ -27,12 +27,12 @@ elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}) endif () if (COPY_HEADERS_COMPILER AND OS_LINUX) - add_custom_target (copy-headers ALL env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh) + add_custom_target (copy-headers env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh) if (USE_INTERNAL_LLVM_LIBRARY) set (CLANG_HEADERS_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/clang/lib/Headers") set (CLANG_HEADERS_DEST "${TMP_HEADERS_DIR}/usr/local/lib/clang/${LLVM_VERSION}/include") # original: ${LLVM_LIBRARY_OUTPUT_INTDIR}/clang/${CLANG_VERSION}/include - add_custom_target (copy-headers-clang ALL ${CMAKE_COMMAND} -E make_directory ${CLANG_HEADERS_DEST} && ${CMAKE_COMMAND} -E copy_if_different ${CLANG_HEADERS_DIR}/* ${CLANG_HEADERS_DEST} ) + add_custom_target (copy-headers-clang ${CMAKE_COMMAND} -E make_directory ${CLANG_HEADERS_DEST} && ${CMAKE_COMMAND} -E copy_if_different ${CLANG_HEADERS_DIR}/* ${CLANG_HEADERS_DEST} ) add_dependencies (copy-headers copy-headers-clang) endif () endif () diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index b88996bd6f9..b4dcaa49410 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -1,8 +1,7 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) -add_headers_and_sources(clickhouse_functions .) add_headers_and_sources(clickhouse_functions ./GatherUtils) -add_headers_and_sources(clickhouse_functions ./Conditional) +add_headers_and_sources(clickhouse_functions .) list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp) diff --git a/dbms/src/Functions/FunctionBinaryArithmetic.h b/dbms/src/Functions/FunctionBinaryArithmetic.h index a668c43b122..12c306cdf34 100644 --- a/dbms/src/Functions/FunctionBinaryArithmetic.h +++ b/dbms/src/Functions/FunctionBinaryArithmetic.h @@ -11,13 +11,14 @@ #include #include #include -#include -#include +#include "IFunction.h" +#include "FunctionHelpers.h" +#include "intDiv.h" +#include "castTypeToEither.h" +#include "FunctionFactory.h" #include #include #include -#include -#include #include #if USE_EMBEDDED_COMPILER diff --git a/dbms/src/Functions/trim.cpp b/dbms/src/Functions/trim.cpp index 0484f369361..ff8c7e536ce 100644 --- a/dbms/src/Functions/trim.cpp +++ b/dbms/src/Functions/trim.cpp @@ -92,7 +92,9 @@ private: { #if __SSE4_2__ /// skip whitespace from left in blocks of up to 16 characters - constexpr auto left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT; + + /// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate + enum { left_sse_mode = base_sse_mode | _SIDD_LEAST_SIGNIFICANT }; while (mask == bytes_sse && chars_to_trim_left < size_sse) { const auto chars = _mm_loadu_si128(reinterpret_cast(data + chars_to_trim_left)); @@ -110,7 +112,9 @@ private: const auto trim_right_size = size - chars_to_trim_left; #if __SSE4_2__ /// try to skip whitespace from right in blocks of up to 16 characters - constexpr auto right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT; + + /// Avoid gcc bug: _mm_cmpistri: error: the third argument must be an 8-bit immediate + enum { right_sse_mode = base_sse_mode | _SIDD_MOST_SIGNIFICANT }; const auto trim_right_size_sse = trim_right_size - (trim_right_size % bytes_sse); while (mask == bytes_sse && chars_to_trim_right < trim_right_size_sse) { diff --git a/utils/check-style/check-include-stat b/utils/check-style/check-include-stat index 9dc9b7e7e9a..13588feb44d 100755 --- a/utils/check-style/check-include-stat +++ b/utils/check-style/check-include-stat @@ -16,10 +16,12 @@ sh ${CUR_DIR}check-include > $RESULT_FILE 2>&1 echo Results: echo Top by memory: -cat $RESULT_FILE | sort -rk4 | head -n20 +cat $RESULT_FILE | sort -nrk4 | head -n20 echo Top by time: -cat $RESULT_FILE | sort -rk3 | head -n20 +cat $RESULT_FILE | sort -nrk3 | head -n20 echo Top by includes: -cat $RESULT_FILE | sort -rk2 | head -n20 +cat $RESULT_FILE | sort -nrk2 | head -n20 + +trap "" EXIT From 7b420297edb100a945a84a1e574ae47e804a9e0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 22:25:40 +0300 Subject: [PATCH 155/181] Fixed "unbundled" build #3905 --- CMakeLists.txt | 1 + cmake/find_xxhash.cmake | 10 ++++++++++ dbms/src/Common/config.h.in | 6 ++++-- dbms/src/Functions/FunctionsHashing.cpp | 4 ++++ dbms/src/Functions/FunctionsHashing.h | 18 +++++++++++++++--- 5 files changed, 34 insertions(+), 5 deletions(-) create mode 100644 cmake/find_xxhash.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 11bc21e62ed..62dc19f1734 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -268,6 +268,7 @@ include (cmake/find_odbc.cmake) # openssl, zlib, odbc before poco include (cmake/find_poco.cmake) include (cmake/find_lz4.cmake) +include (cmake/find_xxhash.cmake) include (cmake/find_sparsehash.cmake) include (cmake/find_rt.cmake) include (cmake/find_execinfo.cmake) diff --git a/cmake/find_xxhash.cmake b/cmake/find_xxhash.cmake new file mode 100644 index 00000000000..0b684838306 --- /dev/null +++ b/cmake/find_xxhash.cmake @@ -0,0 +1,10 @@ +if (LZ4_INCLUDE_DIR) + if (NOT EXISTS "${LZ4_INCLUDE_DIR}/xxhash.h") + message (WARNING "LZ4 library does not have XXHash. Support for XXHash will be disabled.") + set (USE_XXHASH 0) + else () + set (USE_XXHASH 1) + endif () +endif () + +message (STATUS "Using xxhash=${USE_XXHASH}") diff --git a/dbms/src/Common/config.h.in b/dbms/src/Common/config.h.in index 302fc33c6b4..624c87b91b5 100644 --- a/dbms/src/Common/config.h.in +++ b/dbms/src/Common/config.h.in @@ -9,11 +9,13 @@ #cmakedefine01 USE_RDKAFKA #cmakedefine01 USE_CAPNP #cmakedefine01 USE_EMBEDDED_COMPILER -#cmakedefine01 LLVM_HAS_RTTI #cmakedefine01 USE_POCO_SQLODBC #cmakedefine01 USE_POCO_DATAODBC #cmakedefine01 USE_POCO_MONGODB #cmakedefine01 USE_POCO_NETSSL -#cmakedefine01 CLICKHOUSE_SPLIT_BINARY #cmakedefine01 USE_BASE64 #cmakedefine01 USE_HDFS +#cmakedefine01 USE_XXHASH + +#cmakedefine01 CLICKHOUSE_SPLIT_BINARY +#cmakedefine01 LLVM_HAS_RTTI diff --git a/dbms/src/Functions/FunctionsHashing.cpp b/dbms/src/Functions/FunctionsHashing.cpp index 39950d91d70..787fd80ae08 100644 --- a/dbms/src/Functions/FunctionsHashing.cpp +++ b/dbms/src/Functions/FunctionsHashing.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB @@ -27,7 +28,10 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + +#if USE_XXHASH factory.registerFunction(); factory.registerFunction(); +#endif } } diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 28870be3fe4..bce3c27b24f 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -7,7 +7,11 @@ #include #include #include -#include + +#include +#if USE_XXHASH + #include +#endif #include @@ -403,6 +407,9 @@ struct ImplMetroHash64 static constexpr bool use_int_hash_for_pods = true; }; + +#if USE_XXHASH + struct ImplXxHash32 { static constexpr auto name = "xxHash32"; @@ -441,6 +448,8 @@ struct ImplXxHash64 static constexpr bool use_int_hash_for_pods = false; }; +#endif + template class FunctionStringHashFixedString : public IFunction @@ -1064,9 +1073,12 @@ using FunctionMurmurHash2_64 = FunctionAnyHash; using FunctionMurmurHash3_32 = FunctionAnyHash; using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; -using FunctionXxHash32 = FunctionAnyHash; -using FunctionXxHash64 = FunctionAnyHash; using FunctionJavaHash = FunctionAnyHash; using FunctionHiveHash = FunctionAnyHash; +#if USE_XXHASH + using FunctionXxHash32 = FunctionAnyHash; + using FunctionXxHash64 = FunctionAnyHash; +#endif + } From 86c2d0f3290280c38b5fbbcba0f881d359ea6b78 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Dec 2018 23:01:17 +0300 Subject: [PATCH 156/181] Fixed buffer overflow in function addDays [#CLICKHOUSE-2] --- libs/libcommon/include/common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 55a94f3733a..aa3fb4b6f1d 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -517,7 +517,7 @@ public: inline time_t addDays(time_t t, Int64 delta) const { - size_t index = findIndex(t); + UInt16 index = findIndex(t); /// Using UInt16 to possibly overflow within valid range. time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); index += delta; From 986f96ada967c4c1459d3d777efc2a9db81c91a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 00:37:42 +0300 Subject: [PATCH 157/181] Removed redundand code #3785 --- dbms/programs/server/Server.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index eda18809d66..499f233ff28 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -195,7 +195,6 @@ int Server::main(const std::vector & /*args*/) /// Check that the process' user id matches the owner of the data. const auto effective_user_id = geteuid(); struct stat statbuf; - const auto effective_user = getUserName(effective_user_id); if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) { const auto effective_user = getUserName(effective_user_id); From 8367c99720415e672051f9f05ce230b6c7345b68 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 00:38:19 +0300 Subject: [PATCH 158/181] DateLUT: simpler, safer and more efficient [#CLICKHOUSE-2] --- libs/libcommon/include/common/DateLUTImpl.h | 58 +++++++++------------ libs/libcommon/src/DateLUTImpl.cpp | 6 +-- 2 files changed, 27 insertions(+), 37 deletions(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index aa3fb4b6f1d..45948b55425 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -71,26 +71,22 @@ private: /// We can correctly process only timestamps that less DATE_LUT_MAX (i.e. up to 2105 year inclusively) - inline size_t findIndex(time_t t) const + /// We don't care about overflow. + inline DayNum findIndex(time_t t) const { /// First guess. - size_t guess = t / 86400; - if (guess >= DATE_LUT_MAX_DAY_NUM) - return 0; - if (t >= lut[guess].date && t < lut[guess + 1].date) + DayNum guess(t / 86400); + + /// UTC offset is from -12 to +14 in all known time zones. This requires checking only three indices. + + if (t >= lut[guess].date && t < lut[DayNum(guess + 1)].date) return guess; - for (size_t i = 1;; ++i) - { - if (guess + i >= DATE_LUT_MAX_DAY_NUM) - return 0; - if (t >= lut[guess + i].date && t < lut[guess + i + 1].date) - return guess + i; - if (guess < i) - return 0; - if (t >= lut[guess - i].date && t < lut[guess - i + 1].date) - return guess - i; - } + /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). + if (offset_at_start_of_epoch >= 0) + return DayNum(guess + 1); + + return DayNum(guess - 1); } inline const Values & find(time_t t) const @@ -113,8 +109,8 @@ public: /// Round down to start of monday. inline time_t toFirstDayOfWeek(time_t t) const { - size_t index = findIndex(t); - return lut[index - (lut[index].day_of_week - 1)].date; + DayNum index = findIndex(t); + return lut[DayNum(index - (lut[index].day_of_week - 1))].date; } inline DayNum toFirstDayNumOfWeek(DayNum d) const @@ -130,7 +126,7 @@ public: /// Round down to start of month. inline time_t toFirstDayOfMonth(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); return lut[index - (lut[index].day_of_month - 1)].date; } @@ -147,13 +143,13 @@ public: /// Round down to start of quarter. inline DayNum toFirstDayNumOfQuarter(DayNum d) const { - size_t index = d; + DayNum index = d; size_t month_inside_quarter = (lut[index].month - 1) % 3; - index = index - lut[index].day_of_month; + index -= lut[index].day_of_month; while (month_inside_quarter) { - index = index - lut[index].day_of_month; + index -= lut[index].day_of_month; --month_inside_quarter; } @@ -188,14 +184,14 @@ public: inline time_t toFirstDayOfNextMonth(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); index += 32 - lut[index].day_of_month; return lut[index - (lut[index].day_of_month - 1)].date; } inline time_t toFirstDayOfPrevMonth(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); index -= lut[index].day_of_month; return lut[index - (lut[index].day_of_month - 1)].date; } @@ -213,7 +209,7 @@ public: inline UInt8 daysInMonth(UInt16 year, UInt8 month) const { /// 32 makes arithmetic more simple. - auto any_day_of_month = years_lut[year - DATE_LUT_MIN_YEAR] + 32 * (month - 1); + DayNum any_day_of_month = DayNum(years_lut[year - DATE_LUT_MIN_YEAR] + 32 * (month - 1)); return lut[any_day_of_month].days_in_month; } @@ -221,12 +217,12 @@ public: */ inline time_t toDateAndShift(time_t t, Int32 days) const { - return lut[findIndex(t) + days].date; + return lut[DayNum(findIndex(t) + days)].date; } inline time_t toTime(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); if (unlikely(index == 0)) return t + offset_at_start_of_epoch; @@ -241,7 +237,7 @@ public: inline unsigned toHour(time_t t) const { - size_t index = findIndex(t); + DayNum index = findIndex(t); /// If it is not 1970 year (findIndex found nothing appropriate), /// than limit number of hours to avoid insane results like 1970-01-01 89:28:15 @@ -301,7 +297,7 @@ public: * because the same calendar day starts/ends at different timestamps in different time zones) */ - inline DayNum toDayNum(time_t t) const { return static_cast(findIndex(t)); } + inline DayNum toDayNum(time_t t) const { return findIndex(t); } inline time_t fromDayNum(DayNum d) const { return lut[d].date; } inline time_t toDate(DayNum d) const { return lut[d].date; } @@ -517,7 +513,7 @@ public: inline time_t addDays(time_t t, Int64 delta) const { - UInt16 index = findIndex(t); /// Using UInt16 to possibly overflow within valid range. + DayNum index = findIndex(t); time_t time_offset = toHour(t) * 3600 + toMinute(t) * 60 + toSecond(t); index += delta; @@ -687,6 +683,4 @@ public: return s; } - - inline bool isOffsetWholeNumberOfHoursEveryTime() const { return offset_is_whole_number_of_hours_everytime; } }; diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index 76afcd548c2..dfe0cdc0760 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -56,7 +56,6 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) cctz::time_zone::absolute_lookup start_of_epoch_lookup = cctz_time_zone.lookup(std::chrono::system_clock::from_time_t(start_of_day)); offset_at_start_of_epoch = start_of_epoch_lookup.offset; - offset_is_whole_number_of_hours_everytime = true; cctz::civil_day date{1970, 1, 1}; @@ -84,9 +83,6 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) values.time_at_offset_change = 0; values.amount_of_offset_change = 0; - if (start_of_day % 3600) - offset_is_whole_number_of_hours_everytime = false; - /// If UTC offset was changed in previous day. if (i != 0) { @@ -129,7 +125,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) /// Fill excessive part of lookup table. This is needed only to simplify handling of overflow cases. while (i < DATE_LUT_SIZE) { - lut[i] = lut[0]; + lut[i] = lut[DATE_LUT_MAX_DAY_NUM]; ++i; } From fbee51bb7d00f6659faa0b643102080bb99eeca8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 00:37:42 +0300 Subject: [PATCH 159/181] Removed redundand code #3785 --- dbms/programs/server/Server.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index eda18809d66..499f233ff28 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -195,7 +195,6 @@ int Server::main(const std::vector & /*args*/) /// Check that the process' user id matches the owner of the data. const auto effective_user_id = geteuid(); struct stat statbuf; - const auto effective_user = getUserName(effective_user_id); if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) { const auto effective_user = getUserName(effective_user_id); From 344d610ef04c3a3b7d31ea3c6901c91850307537 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 00:45:28 +0300 Subject: [PATCH 160/181] DateLUT: simpler, safer and more efficient [#CLICKHOUSE-2] --- libs/libcommon/src/DateLUTImpl.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index dfe0cdc0760..3f812accb48 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -56,6 +56,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) cctz::time_zone::absolute_lookup start_of_epoch_lookup = cctz_time_zone.lookup(std::chrono::system_clock::from_time_t(start_of_day)); offset_at_start_of_epoch = start_of_epoch_lookup.offset; + offset_is_whole_number_of_hours_everytime = true; cctz::civil_day date{1970, 1, 1}; @@ -83,6 +84,9 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) values.time_at_offset_change = 0; values.amount_of_offset_change = 0; + if (start_of_day % 3600) + offset_is_whole_number_of_hours_everytime = false; + /// If UTC offset was changed in previous day. if (i != 0) { From f09da489f7e6e92d730515ee08c679bdef87d38d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:10:48 +0300 Subject: [PATCH 161/181] Added test #3913 --- .../queries/0_stateless/00809_add_days_segfault.reference | 5 +++++ dbms/tests/queries/0_stateless/00809_add_days_segfault.sql | 6 ++++++ 2 files changed, 11 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00809_add_days_segfault.reference create mode 100644 dbms/tests/queries/0_stateless/00809_add_days_segfault.sql diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference new file mode 100644 index 00000000000..229972f2924 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference @@ -0,0 +1,5 @@ +0 +0 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql new file mode 100644 index 00000000000..8ddc9ba1a27 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql @@ -0,0 +1,6 @@ +SELECT ignore(addDays(toDateTime(0), -1)); +SELECT ignore(subtractDays(toDateTime(0), 1)); +SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); + +SELECT ignore(addDays(toDate(0), -1)); +SELECT ignore(subtractDays(toDate(0), 1)); From ce7c35ac7d7213850b6864e56b4399bf41200a13 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:11:29 +0300 Subject: [PATCH 162/181] Updated documentation about ClickHouse testing [#CLICKHOUSE-2] --- docs/en/development/tests.md | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 5455a234ae3..d9a44f78ea3 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -9,13 +9,13 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `dbms/src/tests/queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from Yandex.Metrica and not available to general public. We tend to use only `stateless` tests and avoid adding new `stateful` tests. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery --testmode`. `.sh` test is a script that is run by itself. To run all tests, use `dbms/tests/clickhouse-test` tool. Look `--help` for the list of possible options. You can simply run all tests or run subset of tests filtered by substring in test name: `./clickhouse-test substring`. The most simple way to invoke functional tests is to copy `clickhouse-client` to `/usr/bin/`, run `clickhouse-server` and then run `./clickhouse-test` from its own directory. -To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. @@ -24,6 +24,11 @@ If you want to use distributed queries in functional tests, you can leverage `re Some tests are marked with `zookeeper`, `shard` or `long` in their names. `zookeeper` is for tests that are using ZooKeeper; `shard` is for tests that requires server to listen `127.0.0.*`; `long` is for tests that run slightly longer that one second. +## Known bugs + +If we know some bugs that can be easily reproduced by functional tests, we place prepared functional tests in `dbms/src/tests/queries/bugs` directory. These tests will be moved to `dbms/src/tests/queries/0_stateless` when bugs are fixed. + + ## Integration Tests Integration tests allow to test ClickHouse in clustered configuration and ClickHouse interaction with other servers like MySQL, Postgres, MongoDB. They are useful to emulate network splits, packet drops, etc. These tests are run under Docker and create multiple containers with various software. @@ -55,7 +60,7 @@ Performance tests are not run on per-commit basis. Results of performance tests Some programs in `tests` directory are not prepared tests, but are test tools. For example, for `Lexer` there is a tool `dbms/src/Parsers/tests/lexer` that just do tokenization of stdin and writes colorized result to stdout. You can use these kind of tools as a code examples and for exploration and manual testing. -You can also place pair of files `.sh` and `.reference` along with the tool to run it on some predefined input - then script result can be compared to `.reference` file. There kind of tests are not automated. +You can also place pair of files `.sh` and `.reference` along with the tool to run it on some predefined input - then script result can be compared to `.reference` file. These kind of tests are not automated. ## Miscellanous Tests @@ -173,7 +178,7 @@ For production builds, gcc is used (it still generates slightly more efficient c ## Sanitizers **Address sanitizer**. -We run functional tests under ASan on per-commit basis. +We run functional and integration tests under ASan on per-commit basis. **Valgrind (Memcheck)**. We run functional tests under Valgrind overnight. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). @@ -185,7 +190,7 @@ We run functional tests under TSan. ClickHouse must pass all tests. Run under TS Currently we still don't use MSan. **Undefined behaviour sanitizer.** -We still don't use UBSan. The only thing to fix is unaligned placement of structs in Arena during aggregation. This is totally fine, we only have to force alignment under UBSan. +We still don't use UBSan on per commit basis. There are some places to fix. **Debug allocator.** You can enable debug version of `tcmalloc` with `DEBUG_TCMALLOC` CMake option. We run tests with debug allocator on per-commit basis. @@ -195,7 +200,9 @@ You will find some additional details in `dbms/tests/instructions/sanitizers.txt ## Fuzzing -As of July 2018 we don't use fuzzing. +We use simple fuzz test to generate random SQL queries and to check that the server doesn't die. Fuzz testing is performed with Address sanitizer. You can find it in `00746_sql_fuzzy.pl`. This test should be run continuously (overnight and longer). + +As of December 2018, we still don't use isolated fuzz testing of library code. ## Security Audit @@ -242,12 +249,12 @@ As of July 2018 we don't track test coverage. ## Test Automation -We run tests with Travis CI (available for general public) and Jenkins (available inside Yandex). +We run tests with Yandex internal CI and job automation system named "Sandbox". We also continue to use Jenkins (available inside Yandex). -In Travis CI due to limit on time and computational power we can afford only subset of functional tests that are run with limited build of ClickHouse (debug version with cut off most of libraries). In about half of runs it still fails to finish in 50 minutes timeout. The only advantage - test results are visible for all external contributors. +Build jobs and tests are run in Sandbox on per commit basis. Resulting packages and test results are published in GitHub and can be downloaded by direct links. Artifacts are stored eternally. When you send a pull request on GitHub, we tag it as "can be tested" and our CI system will build ClickHouse packages (release, debug, with address sanitizer, etc) for you. -In Jenkins we run functional tests for each commit and for each pull request from trusted users; the same under ASan; we also run quorum tests, dictionary tests, Metrica B2B tests. We use Jenkins to prepare and publish releases. Worth to note that we are not happy with Jenkins at all. +We don't use Travis CI due to the limit on time and computational power. -One of our goals is to provide reliable testing infrastructure that will be available to community. +In Jenkins we run dictionary tests, Metrica B2B tests. We use Jenkins to prepare and publish releases. Jenkins is a legacy technology and all jobs will be moved to Sandbox. [Original article](https://clickhouse.yandex/docs/en/development/tests/) From d194ad776691f037cf2d3a15c4441dee3b022fe1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:11:29 +0300 Subject: [PATCH 163/181] Updated documentation about ClickHouse testing [#CLICKHOUSE-2] --- docs/en/development/tests.md | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 5455a234ae3..d9a44f78ea3 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -9,13 +9,13 @@ Each functional test sends one or multiple queries to the running ClickHouse ser Tests are located in `dbms/src/tests/queries` directory. There are two subdirectories: `stateless` and `stateful`. Stateless tests run queries without any preloaded test data - they often create small synthetic datasets on the fly, within the test itself. Stateful tests require preloaded test data from Yandex.Metrica and not available to general public. We tend to use only `stateless` tests and avoid adding new `stateful` tests. -Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery`. `.sh` test is a script that is run by itself. +Each test can be one of two types: `.sql` and `.sh`. `.sql` test is the simple SQL script that is piped to `clickhouse-client --multiquery --testmode`. `.sh` test is a script that is run by itself. To run all tests, use `dbms/tests/clickhouse-test` tool. Look `--help` for the list of possible options. You can simply run all tests or run subset of tests filtered by substring in test name: `./clickhouse-test substring`. The most simple way to invoke functional tests is to copy `clickhouse-client` to `/usr/bin/`, run `clickhouse-server` and then run `./clickhouse-test` from its own directory. -To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. +To add new test, create a `.sql` or `.sh` file in `dbms/src/tests/queries/0_stateless` directory, check it manually and then generate `.reference` file in the following way: `clickhouse-client -n --testmode < 00000_test.sql > 00000_test.reference` or `./00000_test.sh > ./00000_test.reference`. Tests should use (create, drop, etc) only tables in `test` database that is assumed to be created beforehand; also tests can use temporary tables. @@ -24,6 +24,11 @@ If you want to use distributed queries in functional tests, you can leverage `re Some tests are marked with `zookeeper`, `shard` or `long` in their names. `zookeeper` is for tests that are using ZooKeeper; `shard` is for tests that requires server to listen `127.0.0.*`; `long` is for tests that run slightly longer that one second. +## Known bugs + +If we know some bugs that can be easily reproduced by functional tests, we place prepared functional tests in `dbms/src/tests/queries/bugs` directory. These tests will be moved to `dbms/src/tests/queries/0_stateless` when bugs are fixed. + + ## Integration Tests Integration tests allow to test ClickHouse in clustered configuration and ClickHouse interaction with other servers like MySQL, Postgres, MongoDB. They are useful to emulate network splits, packet drops, etc. These tests are run under Docker and create multiple containers with various software. @@ -55,7 +60,7 @@ Performance tests are not run on per-commit basis. Results of performance tests Some programs in `tests` directory are not prepared tests, but are test tools. For example, for `Lexer` there is a tool `dbms/src/Parsers/tests/lexer` that just do tokenization of stdin and writes colorized result to stdout. You can use these kind of tools as a code examples and for exploration and manual testing. -You can also place pair of files `.sh` and `.reference` along with the tool to run it on some predefined input - then script result can be compared to `.reference` file. There kind of tests are not automated. +You can also place pair of files `.sh` and `.reference` along with the tool to run it on some predefined input - then script result can be compared to `.reference` file. These kind of tests are not automated. ## Miscellanous Tests @@ -173,7 +178,7 @@ For production builds, gcc is used (it still generates slightly more efficient c ## Sanitizers **Address sanitizer**. -We run functional tests under ASan on per-commit basis. +We run functional and integration tests under ASan on per-commit basis. **Valgrind (Memcheck)**. We run functional tests under Valgrind overnight. It takes multiple hours. Currently there is one known false positive in `re2` library, see [this article](https://research.swtch.com/sparse). @@ -185,7 +190,7 @@ We run functional tests under TSan. ClickHouse must pass all tests. Run under TS Currently we still don't use MSan. **Undefined behaviour sanitizer.** -We still don't use UBSan. The only thing to fix is unaligned placement of structs in Arena during aggregation. This is totally fine, we only have to force alignment under UBSan. +We still don't use UBSan on per commit basis. There are some places to fix. **Debug allocator.** You can enable debug version of `tcmalloc` with `DEBUG_TCMALLOC` CMake option. We run tests with debug allocator on per-commit basis. @@ -195,7 +200,9 @@ You will find some additional details in `dbms/tests/instructions/sanitizers.txt ## Fuzzing -As of July 2018 we don't use fuzzing. +We use simple fuzz test to generate random SQL queries and to check that the server doesn't die. Fuzz testing is performed with Address sanitizer. You can find it in `00746_sql_fuzzy.pl`. This test should be run continuously (overnight and longer). + +As of December 2018, we still don't use isolated fuzz testing of library code. ## Security Audit @@ -242,12 +249,12 @@ As of July 2018 we don't track test coverage. ## Test Automation -We run tests with Travis CI (available for general public) and Jenkins (available inside Yandex). +We run tests with Yandex internal CI and job automation system named "Sandbox". We also continue to use Jenkins (available inside Yandex). -In Travis CI due to limit on time and computational power we can afford only subset of functional tests that are run with limited build of ClickHouse (debug version with cut off most of libraries). In about half of runs it still fails to finish in 50 minutes timeout. The only advantage - test results are visible for all external contributors. +Build jobs and tests are run in Sandbox on per commit basis. Resulting packages and test results are published in GitHub and can be downloaded by direct links. Artifacts are stored eternally. When you send a pull request on GitHub, we tag it as "can be tested" and our CI system will build ClickHouse packages (release, debug, with address sanitizer, etc) for you. -In Jenkins we run functional tests for each commit and for each pull request from trusted users; the same under ASan; we also run quorum tests, dictionary tests, Metrica B2B tests. We use Jenkins to prepare and publish releases. Worth to note that we are not happy with Jenkins at all. +We don't use Travis CI due to the limit on time and computational power. -One of our goals is to provide reliable testing infrastructure that will be available to community. +In Jenkins we run dictionary tests, Metrica B2B tests. We use Jenkins to prepare and publish releases. Jenkins is a legacy technology and all jobs will be moved to Sandbox. [Original article](https://clickhouse.yandex/docs/en/development/tests/) From 5ccf9e1d54151aca59dd8d9205f312e3ee2c99bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:16:04 +0300 Subject: [PATCH 164/181] Updated test #3913 --- .../queries/0_stateless/00809_add_days_segfault.reference | 1 + .../tests/queries/0_stateless/00809_add_days_segfault.sql | 8 +++++++- dbms/tests/queries/bugs/fuzzy.sql | 4 ---- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference index 229972f2924..f7eb44d66e0 100644 --- a/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.reference @@ -3,3 +3,4 @@ 0 0 0 +0 diff --git a/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql index 8ddc9ba1a27..b087f7bbde5 100644 --- a/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql +++ b/dbms/tests/queries/0_stateless/00809_add_days_segfault.sql @@ -1,6 +1,12 @@ SELECT ignore(addDays(toDateTime(0), -1)); SELECT ignore(subtractDays(toDateTime(0), 1)); -SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); SELECT ignore(addDays(toDate(0), -1)); SELECT ignore(subtractDays(toDate(0), 1)); + +SET send_logs_level = 'none'; + +SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); +SELECT ignore(subtractDays((CAST((-5263074.47) AS DateTime)), -737895)); +SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; -- { serverError 42 } +SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], 'A') AS String))]]); SELECT truncate(895, -16); SELECT notIn([['']], [[NULL]]); -SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); -SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; -SELECT addDays((CAST((96.338) AS DateTime)), -3); From c828afd3db892ef1a4bd86b2b185e2fdcb969a55 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:20:44 +0300 Subject: [PATCH 165/181] Added a test for already fixed bug [#CLICKHOUSE-2] --- .../0_stateless/00810_in_operators_segfault.reference | 0 .../queries/0_stateless/00810_in_operators_segfault.sql | 5 +++++ dbms/tests/queries/bugs/fuzzy.sql | 3 ++- 3 files changed, 7 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference create mode 100644 dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql diff --git a/dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql new file mode 100644 index 00000000000..1fa525eaccc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql @@ -0,0 +1,5 @@ +SET send_logs_level = 'none'; + +SELECT globalNotIn(['"wh'], [NULL]); -- { serverError 53 } +SELECT globalIn([''], [NULL]); -- { serverError 53 } +SELECT notIn([['']], [[NULL]]); -- { serverError 53 } diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index ff82baca47a..541d88eab32 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -1,5 +1,6 @@ SELECT globalNotIn(['"wh'], [NULL]); SELECT globalIn([''], [NULL]) +SELECT notIn([['']], [[NULL]]); + SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); SELECT truncate(895, -16); -SELECT notIn([['']], [[NULL]]); From 7678db6e9da09d40fc9a4f0d8ed3d49a70e24ee1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:20:44 +0300 Subject: [PATCH 166/181] Added a test for already fixed bug [#CLICKHOUSE-2] --- .../0_stateless/00810_in_operators_segfault.reference | 0 .../queries/0_stateless/00810_in_operators_segfault.sql | 5 +++++ dbms/tests/queries/bugs/fuzzy.sql | 3 --- 3 files changed, 5 insertions(+), 3 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference create mode 100644 dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql diff --git a/dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql new file mode 100644 index 00000000000..1fa525eaccc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00810_in_operators_segfault.sql @@ -0,0 +1,5 @@ +SET send_logs_level = 'none'; + +SELECT globalNotIn(['"wh'], [NULL]); -- { serverError 53 } +SELECT globalIn([''], [NULL]); -- { serverError 53 } +SELECT notIn([['']], [[NULL]]); -- { serverError 53 } diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 9a5fd36fbb3..1468ed648b2 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -1,9 +1,6 @@ SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], 'A') AS String))]]); SELECT truncate(895, -16); -SELECT notIn([['']], [[NULL]]); SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; SELECT addDays((CAST((96.338) AS DateTime)), -3); From aca29588c74cbf5a481c75f8de4d9fcb9149ec00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 01:39:49 +0300 Subject: [PATCH 167/181] Added test for already fixed bug [#CLICKHOUSE-2] --- .../queries/0_stateless/00812_prewhere_alias_array.reference | 1 + .../00812_prewhere_alias_array.sql} | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00812_prewhere_alias_array.reference rename dbms/tests/queries/{bugs/prewhere_alias_array.sql => 0_stateless/00812_prewhere_alias_array.sql} (76%) diff --git a/dbms/tests/queries/0_stateless/00812_prewhere_alias_array.reference b/dbms/tests/queries/0_stateless/00812_prewhere_alias_array.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00812_prewhere_alias_array.reference @@ -0,0 +1 @@ +0 diff --git a/dbms/tests/queries/bugs/prewhere_alias_array.sql b/dbms/tests/queries/0_stateless/00812_prewhere_alias_array.sql similarity index 76% rename from dbms/tests/queries/bugs/prewhere_alias_array.sql rename to dbms/tests/queries/0_stateless/00812_prewhere_alias_array.sql index 3281c6ac0c4..0679623194f 100644 --- a/dbms/tests/queries/bugs/prewhere_alias_array.sql +++ b/dbms/tests/queries/0_stateless/00812_prewhere_alias_array.sql @@ -1,4 +1,4 @@ DROP TABLE IF EXISTS test.prewhere; -CREATE TABLE test.prewhere (x Array(UInt64), y ALIAS x, s String) ENGINE = MergeTree ORDER BY tuple() -SELECT count() FROM test.prewhere PREWHERE (length(s) >= 1) = 0 WHERE NOT ignore(y) +CREATE TABLE test.prewhere (x Array(UInt64), y ALIAS x, s String) ENGINE = MergeTree ORDER BY tuple(); +SELECT count() FROM test.prewhere PREWHERE (length(s) >= 1) = 0 WHERE NOT ignore(y); DROP TABLE test.prewhere; From 80f155cfda80cf09086825236e3497a1d3afc8c9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 02:12:30 +0300 Subject: [PATCH 168/181] Fixed overflow in rounding functions with integer argument and large negative scale [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsRound.h | 4 ++-- dbms/tests/queries/0_stateless/00811_garbage.reference | 1 + dbms/tests/queries/0_stateless/00811_garbage.sql | 4 ++++ dbms/tests/queries/bugs/fuzzy.sql | 2 -- libs/libcommon/include/common/intExp.h | 5 +++-- 5 files changed, 10 insertions(+), 6 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00811_garbage.reference create mode 100644 dbms/tests/queries/0_stateless/00811_garbage.sql diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index 3c9be5f15b9..db41a72eadb 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -87,7 +87,7 @@ struct IntegerRoundingComputation return scale; } - static ALWAYS_INLINE T computeImpl(T x, T scale) + static ALWAYS_INLINE T computeImpl(T x, size_t scale) { switch (rounding_mode) { @@ -122,7 +122,7 @@ struct IntegerRoundingComputation } } - static ALWAYS_INLINE T compute(T x, T scale) + static ALWAYS_INLINE T compute(T x, size_t scale) { switch (scale_mode) { diff --git a/dbms/tests/queries/0_stateless/00811_garbage.reference b/dbms/tests/queries/0_stateless/00811_garbage.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00811_garbage.reference @@ -0,0 +1 @@ +0 diff --git a/dbms/tests/queries/0_stateless/00811_garbage.sql b/dbms/tests/queries/0_stateless/00811_garbage.sql new file mode 100644 index 00000000000..65db1c736fa --- /dev/null +++ b/dbms/tests/queries/0_stateless/00811_garbage.sql @@ -0,0 +1,4 @@ +SET send_logs_level = 'none'; + +SELECT truncate(895, -16); +SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); -- { serverError 44 } diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 1468ed648b2..2830fe7c2f9 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -1,6 +1,4 @@ SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], 'A') AS String))]]); -SELECT truncate(895, -16); SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; SELECT addDays((CAST((96.338) AS DateTime)), -3); diff --git a/libs/libcommon/include/common/intExp.h b/libs/libcommon/include/common/intExp.h index 9ecd3f07dd1..8c46d9f26c2 100644 --- a/libs/libcommon/include/common/intExp.h +++ b/libs/libcommon/include/common/intExp.h @@ -32,7 +32,8 @@ inline uint64_t intExp10(int x) return table[x]; } -namespace common { +namespace common +{ inline int exp10_i32(int x) { @@ -123,4 +124,4 @@ inline __int128 exp10_i128(int x) return values[x]; } -} // common +} From 101e36470cf0609c0540af4344898f3a3e0dbd4c Mon Sep 17 00:00:00 2001 From: "Sergey V. Galtsev" Date: Mon, 24 Dec 2018 02:24:27 +0300 Subject: [PATCH 169/181] Remove duplicate line. --- dbms/programs/server/Server.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index eda18809d66..499f233ff28 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -195,7 +195,6 @@ int Server::main(const std::vector & /*args*/) /// Check that the process' user id matches the owner of the data. const auto effective_user_id = geteuid(); struct stat statbuf; - const auto effective_user = getUserName(effective_user_id); if (stat(path.c_str(), &statbuf) == 0 && effective_user_id != statbuf.st_uid) { const auto effective_user = getUserName(effective_user_id); From 0b760ee9e92dd9e834a62d956376d857231803ff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 03:47:11 +0300 Subject: [PATCH 170/181] Fixed overflow in rounding functions with integer argument and large negative scale [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsRound.h | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsRound.h b/dbms/src/Functions/FunctionsRound.h index db41a72eadb..4d8f5edccc3 100644 --- a/dbms/src/Functions/FunctionsRound.h +++ b/dbms/src/Functions/FunctionsRound.h @@ -87,7 +87,7 @@ struct IntegerRoundingComputation return scale; } - static ALWAYS_INLINE T computeImpl(T x, size_t scale) + static ALWAYS_INLINE T computeImpl(T x, T scale) { switch (rounding_mode) { @@ -122,7 +122,7 @@ struct IntegerRoundingComputation } } - static ALWAYS_INLINE T compute(T x, size_t scale) + static ALWAYS_INLINE T compute(T x, T scale) { switch (scale_mode) { @@ -139,7 +139,10 @@ struct IntegerRoundingComputation static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) { - *out = compute(*in, scale); + if (scale > size_t(std::numeric_limits::max())) + *out = 0; + else + *out = compute(*in, scale); } }; From 4f9630fd03fee72208d5dce4103b3157d19f6a6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 05:57:22 +0300 Subject: [PATCH 171/181] Fixed error #3913 --- libs/libcommon/include/common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 45948b55425..8fd015afa15 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -79,7 +79,7 @@ private: /// UTC offset is from -12 to +14 in all known time zones. This requires checking only three indices. - if (t >= lut[guess].date && t < lut[DayNum(guess + 1)].date) + if ((guess == 0 || t >= lut[guess].date) && t < lut[DayNum(guess + 1)].date) return guess; /// Time zones that have offset 0 from UTC do daylight saving time change (if any) towards increasing UTC offset (example: British Standard Time). From 416d6aa788685388104afc83227fcb732ab01a91 Mon Sep 17 00:00:00 2001 From: qianlixiang Date: Mon, 24 Dec 2018 16:17:22 +0800 Subject: [PATCH 172/181] Fixed core dump caused by kill query sync --- dbms/src/Interpreters/InterpreterKillQueryQuery.cpp | 7 +++++-- dbms/src/Parsers/ASTKillQueryQuery.h | 8 ++++++-- dbms/src/Parsers/ParserKillQueryQuery.cpp | 3 ++- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index 7e8d783836a..31535bc4bbe 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -221,8 +221,11 @@ BlockIO InterpreterKillQueryQuery::execute() Block InterpreterKillQueryQuery::getSelectFromSystemProcessesResult() { - String system_processes_query = "SELECT query_id, user, query FROM system.processes WHERE " - + queryToString(static_cast(*query_ptr).where_expression); + String system_processes_query = "SELECT query_id, user, query FROM system.processes"; + auto & where_expression = static_cast(*query_ptr).where_expression; + if (where_expression) + system_processes_query += " WHERE " + queryToString(where_expression); + BlockIO system_processes_io = executeQuery(system_processes_query, context, true); Block res = system_processes_io.in->read(); diff --git a/dbms/src/Parsers/ASTKillQueryQuery.h b/dbms/src/Parsers/ASTKillQueryQuery.h index 7099fbccece..34fdba051d1 100644 --- a/dbms/src/Parsers/ASTKillQueryQuery.h +++ b/dbms/src/Parsers/ASTKillQueryQuery.h @@ -15,8 +15,12 @@ public: ASTPtr clone() const override { auto clone = std::make_shared(*this); - clone->where_expression = where_expression->clone(); - clone->children = {clone->where_expression}; + if (where_expression) + { + clone->where_expression = where_expression->clone(); + clone->children = {clone->where_expression}; + } + return clone; } diff --git a/dbms/src/Parsers/ParserKillQueryQuery.cpp b/dbms/src/Parsers/ParserKillQueryQuery.cpp index 5e674d9da83..56b0d3cd557 100644 --- a/dbms/src/Parsers/ParserKillQueryQuery.cpp +++ b/dbms/src/Parsers/ParserKillQueryQuery.cpp @@ -39,7 +39,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->test = true; query->cluster = cluster_str; - query->children.emplace_back(query->where_expression); + if (query->where_expression) + query->children.emplace_back(query->where_expression); node = std::move(query); return true; } From 184c36d78d5f4949038bc7107449ec067628ed64 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 14:06:35 +0300 Subject: [PATCH 173/181] Fixed error in KILL QUERY #3916 --- dbms/src/Parsers/ASTKillQueryQuery.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/src/Parsers/ASTKillQueryQuery.cpp b/dbms/src/Parsers/ASTKillQueryQuery.cpp index 9e7631eacc3..0c9dba01219 100644 --- a/dbms/src/Parsers/ASTKillQueryQuery.cpp +++ b/dbms/src/Parsers/ASTKillQueryQuery.cpp @@ -13,10 +13,12 @@ void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatS settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY"; formatOnCluster(settings); - settings.ostr << " WHERE " << (settings.hilite ? hilite_none : ""); if (where_expression) + { + settings.ostr << " WHERE " << (settings.hilite ? hilite_none : ""); where_expression->formatImpl(settings, state, frame); + } settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << (test ? "TEST" : (sync ? "SYNC" : "ASYNC")) << (settings.hilite ? hilite_none : ""); } From 7cfa7c3dbe1865d22dde63d0a434806e4597b922 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 14:08:09 +0300 Subject: [PATCH 174/181] KILL QUERY: make WHERE clause mandatory #3916 --- dbms/src/Parsers/ParserKillQueryQuery.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Parsers/ParserKillQueryQuery.cpp b/dbms/src/Parsers/ParserKillQueryQuery.cpp index 56b0d3cd557..29e0b572f83 100644 --- a/dbms/src/Parsers/ParserKillQueryQuery.cpp +++ b/dbms/src/Parsers/ParserKillQueryQuery.cpp @@ -28,7 +28,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect if (p_on.ignore(pos, expected) && !ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) return false; - if (p_where.ignore(pos, expected) && !p_where_expression.parse(pos, query->where_expression, expected)) + if (!p_where.ignore(pos, expected) || !p_where_expression.parse(pos, query->where_expression, expected)) return false; if (p_sync.ignore(pos, expected)) @@ -39,8 +39,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->test = true; query->cluster = cluster_str; - if (query->where_expression) - query->children.emplace_back(query->where_expression); + query->children.emplace_back(query->where_expression); node = std::move(query); return true; } From 1070702168f29704080b94f7a808f4d0a16ab2b4 Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Mon, 24 Dec 2018 15:35:46 +0300 Subject: [PATCH 175/181] Update CatBoostModel.cpp Do not evaluate CatBoost model on empty dataset. --- dbms/src/Interpreters/CatBoostModel.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Interpreters/CatBoostModel.cpp b/dbms/src/Interpreters/CatBoostModel.cpp index 61fc1d19785..bd7c07813d0 100644 --- a/dbms/src/Interpreters/CatBoostModel.cpp +++ b/dbms/src/Interpreters/CatBoostModel.cpp @@ -331,6 +331,9 @@ private: auto result = ColumnFloat64::create(column_size); auto result_buf = result->getData().data(); + if (!column_size) + return result; + /// Prepare float features. PODArray float_features(column_size); auto float_features_buf = float_features.data(); From 99c73209a942db1d5a1c76a9c052dbb637768324 Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Mon, 24 Dec 2018 15:37:46 +0300 Subject: [PATCH 176/181] Update build_catboost.sh Fix build script for CatBoost models. --- dbms/tests/external_models/catboost/data/build_catboost.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/external_models/catboost/data/build_catboost.sh b/dbms/tests/external_models/catboost/data/build_catboost.sh index 080c5bcb245..50a3fb43ef8 100755 --- a/dbms/tests/external_models/catboost/data/build_catboost.sh +++ b/dbms/tests/external_models/catboost/data/build_catboost.sh @@ -12,6 +12,6 @@ cd $DIR ln -sf "${DIR}/build/lib/catboost/libs/model_interface/libcatboostmodel.so" libcatboostmodel.so cd "${DIR}/catboost/catboost/python-package/catboost" -../../../ya make -r -DUSE_ARCADIA_PYTHON=no -DPYTHON_CONFIG=python2-config -j4 +../../../ya make -r -DUSE_ARCADIA_PYTHON=no -DOS_SDK=local -DPYTHON_CONFIG=python2-config -j4 cd $DIR ln -sf "${DIR}/catboost/catboost/python-package" python-package From a380cb11f876125ebed8734871d1fba915b8c296 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 24 Dec 2018 15:45:50 +0300 Subject: [PATCH 177/181] CLICKHOUSE-4223 Fix .deb install bad message --- debian/clickhouse-server.postinst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index e17c47df6dd..80b25d47f62 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -76,7 +76,7 @@ Please fix this and reinstall this package." >&2 fi if [ -d ${CLICKHOUSE_CONFDIR} ]; then - rm -v ${CLICKHOUSE_CONFDIR}/*-preprocessed.xml ||: + rm -fv ${CLICKHOUSE_CONFDIR}/*-preprocessed.xml ||: fi ln -s ${CLICKHOUSE_DATADIR}/preprocessed_configs ${CLICKHOUSE_CONFDIR}/preprocessed ||: From 1dffa56073fc835bbfa05de8da9fdb84f16d000c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 15:47:24 +0300 Subject: [PATCH 178/181] Fixed test #3913 --- .../0_stateless/00569_parse_date_time_best_effort.reference | 1 - .../queries/0_stateless/00569_parse_date_time_best_effort.sql | 1 - 2 files changed, 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference index 4b209d6a90f..8638c0b707f 100644 --- a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference +++ b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference @@ -7,7 +7,6 @@ s a b 02/01/17 010203 MSK 2017-01-01 22:02:03 2017-01-01 22:02:03 02/01/17 010203 MSK+0100 2017-01-01 21:02:03 2017-01-01 21:02:03 02/01/17 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 -020117 010203 UTC+0300 1970-01-01 04:30:19 1970-01-01 04:30:19 02/01/17 010203Z 2017-01-02 01:02:03 2017-01-02 01:02:03 02/01/1970 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 02/01/70 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 diff --git a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql index 35e0d248585..5f71efa1485 100644 --- a/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql +++ b/dbms/tests/queries/0_stateless/00569_parse_date_time_best_effort.sql @@ -12,7 +12,6 @@ FROM '02/01/17 010203 MSK', '02/01/17 010203 MSK+0100', '02/01/17 010203 UTC+0300', -'020117 010203 UTC+0300', '02/01/17 010203Z', '02/01/1970 010203Z', '02/01/70 010203Z', From ce1067410718ca9c818d6b5705a875addcaee5b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 16:25:48 +0300 Subject: [PATCH 179/181] Fixed weird error in PR #3870 --- dbms/src/Functions/FunctionsConversion.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 0e0bf218db3..5ca141a64a0 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1161,6 +1161,10 @@ struct ToIntMonotonicity checkAndGetDataType>(&type)) return { true, true, true }; + /// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. + if (left.isNull() || right.isNull()) + return {}; + /// If converting from float, for monotonicity, arguments must fit in range of result type. if (WhichDataType(type).isFloat()) { From 66a0c1691db8f830402289dbae01108fd5b4e046 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 21:28:41 +0300 Subject: [PATCH 180/181] Added more formats to "parseDateTimeBestEffort" function [#CLICKHOUSE-2] --- dbms/src/IO/parseDateTimeBestEffort.cpp | 56 ++++++++++++++----- dbms/src/IO/parseDateTimeBestEffort.h | 2 +- ...parse_date_time_best_effort_more.reference | 21 +++++++ ...00813_parse_date_time_best_effort_more.sql | 28 ++++++++++ 4 files changed, 91 insertions(+), 16 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference create mode 100644 dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql diff --git a/dbms/src/IO/parseDateTimeBestEffort.cpp b/dbms/src/IO/parseDateTimeBestEffort.cpp index ff9b8d067a9..4aea4d621c2 100644 --- a/dbms/src/IO/parseDateTimeBestEffort.cpp +++ b/dbms/src/IO/parseDateTimeBestEffort.cpp @@ -86,6 +86,25 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date bool is_pm = false; + auto read_alpha_month = [&month] (const auto & alpha) + { + if (0 == strncasecmp(alpha, "Jan", 3)) month = 1; + else if (0 == strncasecmp(alpha, "Feb", 3)) month = 2; + else if (0 == strncasecmp(alpha, "Mar", 3)) month = 3; + else if (0 == strncasecmp(alpha, "Apr", 3)) month = 4; + else if (0 == strncasecmp(alpha, "May", 3)) month = 5; + else if (0 == strncasecmp(alpha, "Jun", 3)) month = 6; + else if (0 == strncasecmp(alpha, "Jul", 3)) month = 7; + else if (0 == strncasecmp(alpha, "Aug", 3)) month = 8; + else if (0 == strncasecmp(alpha, "Sep", 3)) month = 9; + else if (0 == strncasecmp(alpha, "Oct", 3)) month = 10; + else if (0 == strncasecmp(alpha, "Nov", 3)) month = 11; + else if (0 == strncasecmp(alpha, "Dec", 3)) month = 12; + else + return false; + return true; + }; + while (!in.eof()) { char digits[14]; @@ -205,6 +224,10 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date /// hh - only if already have day of month /// DD/MM/YYYY /// DD/MM/YY + /// DD.MM.YYYY + /// DD.MM.YY + /// DD-MM-YYYY + /// DD-MM-YY /// DD UInt8 hour_or_day_of_month = 0; @@ -244,7 +267,7 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date return on_error("Cannot read DateTime: unexpected number of decimal digits after hour and minute: " + toString(num_digits), ErrorCodes::CANNOT_PARSE_DATETIME); } } - else if (checkChar('/', in)) + else if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) { if (day_of_month) return on_error("Cannot read DateTime: day of month is duplicated", ErrorCodes::CANNOT_PARSE_DATETIME); @@ -260,10 +283,23 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date readDecimalNumber<2>(month, digits); else if (num_digits == 1) readDecimalNumber<1>(month, digits); + else if (num_digits == 0) + { + /// Month in alphabetical form + + char alpha[9]; /// The longest month name: September + size_t num_alpha = readAlpha(alpha, sizeof(alpha), in); + + if (num_alpha < 3) + return on_error("Cannot read DateTime: unexpected number of alphabetical characters after day of month: " + toString(num_alpha), ErrorCodes::CANNOT_PARSE_DATETIME); + + if (!read_alpha_month(alpha)) + return on_error("Cannot read DateTime: alphabetical characters after day of month don't look like month: " + std::string(alpha, 3), ErrorCodes::CANNOT_PARSE_DATETIME); + } else return on_error("Cannot read DateTime: unexpected number of decimal digits after day of month: " + toString(num_digits), ErrorCodes::CANNOT_PARSE_DATETIME); - if (checkChar('/', in)) + if (checkChar('/', in) || checkChar('.', in) || checkChar('-', in)) { if (year) return on_error("Cannot read DateTime: year component is duplicated", ErrorCodes::CANNOT_PARSE_DATETIME); @@ -401,19 +437,9 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date { bool has_day_of_week = false; - if (0 == strncasecmp(alpha, "Jan", 3)) month = 1; - else if (0 == strncasecmp(alpha, "Feb", 3)) month = 2; - else if (0 == strncasecmp(alpha, "Mar", 3)) month = 3; - else if (0 == strncasecmp(alpha, "Apr", 3)) month = 4; - else if (0 == strncasecmp(alpha, "May", 3)) month = 5; - else if (0 == strncasecmp(alpha, "Jun", 3)) month = 6; - else if (0 == strncasecmp(alpha, "Jul", 3)) month = 7; - else if (0 == strncasecmp(alpha, "Aug", 3)) month = 8; - else if (0 == strncasecmp(alpha, "Sep", 3)) month = 9; - else if (0 == strncasecmp(alpha, "Oct", 3)) month = 10; - else if (0 == strncasecmp(alpha, "Nov", 3)) month = 11; - else if (0 == strncasecmp(alpha, "Dec", 3)) month = 12; - + if (read_alpha_month(alpha)) + { + } else if (0 == strncasecmp(alpha, "UTC", 3)) has_time_zone_offset = true; else if (0 == strncasecmp(alpha, "GMT", 3)) has_time_zone_offset = true; else if (0 == strncasecmp(alpha, "MSK", 3)) { has_time_zone_offset = true; time_zone_offset_hour = 3; } diff --git a/dbms/src/IO/parseDateTimeBestEffort.h b/dbms/src/IO/parseDateTimeBestEffort.h index 65c822ff331..dc416a750c5 100644 --- a/dbms/src/IO/parseDateTimeBestEffort.h +++ b/dbms/src/IO/parseDateTimeBestEffort.h @@ -34,7 +34,7 @@ class ReadBuffer; * YYYYMM - 6 digits is a year, month if year was not already read * hhmmss - 6 digits is a time if year was already read * - * .nnnnnnn - any number of digits after point is fractional part of second, if it is not YYYY.MM.DD + * .nnnnnnn - any number of digits after point is fractional part of second, if it is not YYYY.MM.DD or DD.MM.YYYY * * T - means that time will follow * diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference new file mode 100644 index 00000000000..3ec7da1c47e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference @@ -0,0 +1,21 @@ +s a b + +24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 +24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 +24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 +24.12.18 010203 2018-12-24 01:02:03 2018-12-24 01:02:03 +24.12.18 01:02:03 2018-12-24 01:02:03 2018-12-24 01:02:03 +24.DEC.18T01:02:03.000+0300 2018-12-23 22:02:03 2018-12-23 22:02:03 diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql new file mode 100644 index 00000000000..4a24d73f55a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql @@ -0,0 +1,28 @@ +SELECT + s, + parseDateTimeBestEffortOrNull(s, 'UTC') AS a, + parseDateTimeBestEffortOrZero(s, 'UTC') AS b +FROM +( + SELECT arrayJoin([ +'24.12.2018', +'24-12-2018', +'24.12.18', +'24-12-18', +'24-Dec-18', +'24/DEC/18', +'24/DEC/2018', +'01-OCT-2015', +'24.12.2018', +'24-12-2018', +'24.12.18', +'24-12-18', +'24-Dec-18', +'24/DEC/18', +'24/DEC/2018', +'01-OCT-2015', +'24.12.18 010203', +'24.12.18 01:02:03', +'24.DEC.18T01:02:03.000+0300' +]) AS s) +FORMAT PrettySpaceNoEscapes; From 41e3bf55180dc735dd1b7ad202501ae109bcad45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Dec 2018 21:31:43 +0300 Subject: [PATCH 181/181] Added more formats to "parseDateTimeBestEffort" function [#CLICKHOUSE-2] --- .../00813_parse_date_time_best_effort_more.reference | 1 + .../0_stateless/00813_parse_date_time_best_effort_more.sql | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference index 3ec7da1c47e..921400838d0 100644 --- a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference @@ -19,3 +19,4 @@ s a b 24.12.18 010203 2018-12-24 01:02:03 2018-12-24 01:02:03 24.12.18 01:02:03 2018-12-24 01:02:03 2018-12-24 01:02:03 24.DEC.18T01:02:03.000+0300 2018-12-23 22:02:03 2018-12-23 22:02:03 +01-September-2018 11:22 2018-09-01 11:22:00 2018-09-01 11:22:00 diff --git a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql index 4a24d73f55a..1e3b24e60c0 100644 --- a/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql +++ b/dbms/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.sql @@ -23,6 +23,7 @@ FROM '01-OCT-2015', '24.12.18 010203', '24.12.18 01:02:03', -'24.DEC.18T01:02:03.000+0300' +'24.DEC.18T01:02:03.000+0300', +'01-September-2018 11:22' ]) AS s) FORMAT PrettySpaceNoEscapes;