From 2a2c35e4c11fe1ef245577cb8c1f2cc1db5f9284 Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 25 May 2023 18:14:16 +0300 Subject: [PATCH 001/159] Fix changed IP for https session --- .../Net/include/Poco/Net/HTTPClientSession.h | 3 + src/IO/HTTPCommon.cpp | 29 ++++-- .../configs/listen_host.xml | 3 + .../test_https_replication/test_change_ip.py | 96 +++++++++++++++++++ 4 files changed, 123 insertions(+), 8 deletions(-) create mode 100644 tests/integration/test_https_replication/configs/listen_host.xml create mode 100644 tests/integration/test_https_replication/test_change_ip.py diff --git a/base/poco/Net/include/Poco/Net/HTTPClientSession.h b/base/poco/Net/include/Poco/Net/HTTPClientSession.h index 6f67918d2c8..d495d662f75 100644 --- a/base/poco/Net/include/Poco/Net/HTTPClientSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPClientSession.h @@ -127,6 +127,9 @@ namespace Net void setResolvedHost(std::string resolved_host) { _resolved_host.swap(resolved_host); } + std::string getResolvedHost() const { return _resolved_host; } + /// Returns the resolved IP address of the target HTTP server. + Poco::UInt16 getPort() const; /// Returns the port number of the target HTTP server. diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 4bea646a42b..3ec9b3d0a83 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -68,7 +68,8 @@ namespace if (https) { #if USE_SSL - String resolved_host = resolve_host ? DNSResolver::instance().resolveHost(host).toString() : host; + /// Cannot resolve host in advance, otherwise SNI won't work in Poco. + /// For more information about SNI, see the https://en.wikipedia.org/wiki/Server_Name_Indication auto https_session = std::make_shared(host, port); if (resolve_host) https_session->setResolvedHost(DNSResolver::instance().resolveHost(host).toString()); @@ -184,6 +185,24 @@ namespace std::mutex mutex; std::unordered_map endpoints_pool; + void updateHostIfIpChanged(Entry & session, const String & new_ip) + { + const auto old_ip = session->getResolvedHost().empty() ? session->getHost() : session->getResolvedHost(); + + if (new_ip != old_ip) + { + session->reset(); + if (session->getResolvedHost().empty()) + { + session->setHost(new_ip); + } + else + { + session->setResolvedHost(new_ip); + } + } + } + protected: HTTPSessionPool() = default; @@ -238,13 +257,7 @@ namespace if (resolve_host) { - /// Host can change IP - const auto ip = DNSResolver::instance().resolveHost(host).toString(); - if (ip != session->getHost()) - { - session->reset(); - session->setHost(ip); - } + updateHostIfIpChanged(session, DNSResolver::instance().resolveHost(host).toString()); } } /// Reset the message, once it has been printed, diff --git a/tests/integration/test_https_replication/configs/listen_host.xml b/tests/integration/test_https_replication/configs/listen_host.xml new file mode 100644 index 00000000000..f94e5c88568 --- /dev/null +++ b/tests/integration/test_https_replication/configs/listen_host.xml @@ -0,0 +1,3 @@ + + :: + diff --git a/tests/integration/test_https_replication/test_change_ip.py b/tests/integration/test_https_replication/test_change_ip.py new file mode 100644 index 00000000000..14fe5351c8d --- /dev/null +++ b/tests/integration/test_https_replication/test_change_ip.py @@ -0,0 +1,96 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +""" +Both ssl_conf.xml and no_ssl_conf.xml have the same port +""" + + +def _fill_nodes(nodes, shard): + for node in nodes: + node.query( + """ + CREATE DATABASE test; + + CREATE TABLE test_table(date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test{shard}/replicated', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; + """.format( + shard=shard, replica=node.name + ) + ) + + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=[ + "configs/remote_servers.xml", + "configs/listen_host.xml", + "configs/ssl_conf.xml", + "configs/server.crt", + "configs/server.key", + "configs/dhparam.pem", + ], + with_zookeeper=True, + ipv6_address="2001:3984:3989::1:1111", +) +node2 = cluster.add_instance( + "node2", + main_configs=[ + "configs/remote_servers.xml", + "configs/listen_host.xml", + "configs/ssl_conf.xml", + "configs/server.crt", + "configs/server.key", + "configs/dhparam.pem", + ], + with_zookeeper=True, + ipv6_address="2001:3984:3989::1:1112", +) + + +@pytest.fixture(scope="module") +def both_https_cluster(): + try: + cluster.start() + + _fill_nodes([node1, node2], 1) + + yield cluster + + finally: + cluster.shutdown() + + +def test_replication_when_node_ip_changed(both_https_cluster): + """ + Test for a bug when replication over HTTPS stops working when the IP of the source replica was changed. + + node1 is a source node + node2 fethes data from node1 + """ + node1.query("truncate table test_table") + node2.query("truncate table test_table") + + # First we check, that normal replication works + node1.query( + "INSERT INTO test_table VALUES ('2022-10-01', 1), ('2022-10-02', 2), ('2022-10-03', 3)" + ) + assert node1.query("SELECT count(*) from test_table") == "3\n" + assert_eq_with_retry(node2, "SELECT count(*) from test_table", "3") + + # We change source node ip + cluster.restart_instance_with_ip_change(node1, "2001:3984:3989::1:7777") + + # Put some data to source node1 + node1.query( + "INSERT INTO test_table VALUES ('2018-10-01', 4), ('2018-10-02', 4), ('2018-10-03', 6)" + ) + # Check that data is placed on node1 + assert node1.query("SELECT count(*) from test_table") == "6\n" + + # drop DNS cache + node2.query("SYSTEM DROP DNS CACHE") + # Data is fetched + assert_eq_with_retry(node2, "SELECT count(*) from test_table", "6") From 806b7fc18b0e526f39f6b9f04faa2ee90d4136ee Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sat, 27 May 2023 17:39:21 +0800 Subject: [PATCH 002/159] fix datetime64 index querying --- src/Interpreters/convertFieldToType.cpp | 8 ++++++++ .../1_stateful/00178_query_datetime64_index.reference | 1 + tests/queries/1_stateful/00178_query_datetime64_index.sql | 8 ++++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/1_stateful/00178_query_datetime64_index.reference create mode 100644 tests/queries/1_stateful/00178_query_datetime64_index.sql diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index dc61e748db6..9d82cade814 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -192,6 +192,14 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } + else if(which_type.isDateTime64() && which_from_type.isDate()) { + const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + } + else if(which_type.isDateTime64() && which_from_type.isDate()) { + const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { if (which_type.isUInt8()) return convertNumericType(src, type); diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.reference b/tests/queries/1_stateful/00178_query_datetime64_index.reference new file mode 100644 index 00000000000..3d26cfc1d66 --- /dev/null +++ b/tests/queries/1_stateful/00178_query_datetime64_index.reference @@ -0,0 +1 @@ +2023-05-27 00:00:00.000 diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql new file mode 100644 index 00000000000..0e46401c2d3 --- /dev/null +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS datetime64_index_tbl; + +CREATE TABLE datetime64_index_tbl(ts DateTime64(3, 'UTC')) ENGINE=MergeTree ORDER BY ts; +INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', 3, 'UTC')); + +SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); + +DROP TABLE datetime64_index_tbl; \ No newline at end of file From 41c599b86252d7219adc8e0adc455d93d5e7a5ee Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sat, 27 May 2023 17:59:41 +0800 Subject: [PATCH 003/159] fix date32 --- src/Interpreters/convertFieldToType.cpp | 2 +- tests/queries/1_stateful/00178_query_datetime64_index.reference | 1 + tests/queries/1_stateful/00178_query_datetime64_index.sql | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 9d82cade814..9bea9f6851f 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -196,7 +196,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } - else if(which_type.isDateTime64() && which_from_type.isDate()) { + else if(which_type.isDateTime64() && which_from_type.isDate32()) { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.reference b/tests/queries/1_stateful/00178_query_datetime64_index.reference index 3d26cfc1d66..f24667b36da 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.reference +++ b/tests/queries/1_stateful/00178_query_datetime64_index.reference @@ -1 +1,2 @@ 2023-05-27 00:00:00.000 +2023-05-27 00:00:00.000 diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql index 0e46401c2d3..1e1d1561af7 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.sql +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -4,5 +4,6 @@ CREATE TABLE datetime64_index_tbl(ts DateTime64(3, 'UTC')) ENGINE=MergeTree ORDE INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', 3, 'UTC')); SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); +SELECT ts from datetime64_index_tbl where ts < toDate32('2023-05-28'); DROP TABLE datetime64_index_tbl; \ No newline at end of file From a8122861213071026430af636801587679b3e710 Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sat, 27 May 2023 18:04:19 +0800 Subject: [PATCH 004/159] new line eof --- tests/queries/1_stateful/00178_query_datetime64_index.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql index 1e1d1561af7..68542bc0524 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.sql +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -6,4 +6,4 @@ INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); SELECT ts from datetime64_index_tbl where ts < toDate32('2023-05-28'); -DROP TABLE datetime64_index_tbl; \ No newline at end of file +DROP TABLE datetime64_index_tbl; From 93415789d7e69c2874171941f56eb92a6bc80bbf Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sun, 28 May 2023 10:40:34 +0800 Subject: [PATCH 005/159] format --- src/Interpreters/convertFieldToType.cpp | 4 ++-- tests/queries/1_stateful/00178_query_datetime64_index.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 9bea9f6851f..ba02eefabc9 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -192,11 +192,11 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } - else if(which_type.isDateTime64() && which_from_type.isDate()) { + else if (which_type.isDateTime64() && which_from_type.isDate()) { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } - else if(which_type.isDateTime64() && which_from_type.isDate32()) { + else if (which_type.isDateTime64() && which_from_type.isDate32()) { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } diff --git a/tests/queries/1_stateful/00178_query_datetime64_index.sql b/tests/queries/1_stateful/00178_query_datetime64_index.sql index 68542bc0524..a3fb594db73 100644 --- a/tests/queries/1_stateful/00178_query_datetime64_index.sql +++ b/tests/queries/1_stateful/00178_query_datetime64_index.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS datetime64_index_tbl; CREATE TABLE datetime64_index_tbl(ts DateTime64(3, 'UTC')) ENGINE=MergeTree ORDER BY ts; INSERT INTO datetime64_index_tbl(ts) VALUES(toDateTime64('2023-05-27 00:00:00', 3, 'UTC')); -SELECT ts from datetime64_index_tbl where ts < toDate('2023-05-28'); -SELECT ts from datetime64_index_tbl where ts < toDate32('2023-05-28'); +SELECT ts FROM datetime64_index_tbl WHERE ts < toDate('2023-05-28'); +SELECT ts FROM datetime64_index_tbl WHERE ts < toDate32('2023-05-28'); DROP TABLE datetime64_index_tbl; From 0d873d50a33d4837d4010be0385df23c8be9b204 Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Sun, 28 May 2023 10:49:36 +0800 Subject: [PATCH 006/159] format --- src/Interpreters/convertFieldToType.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index ba02eefabc9..79251d944c8 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -192,11 +192,13 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID { return static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); } - else if (which_type.isDateTime64() && which_from_type.isDate()) { + else if (which_type.isDateTime64() && which_from_type.isDate()) + { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } - else if (which_type.isDateTime64() && which_from_type.isDate32()) { + else if (which_type.isDateTime64() && which_from_type.isDate32()) + { const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); } From 954e76cb7a432081df8865a2988749fec3b950e4 Mon Sep 17 00:00:00 2001 From: Lucas Chang Date: Tue, 30 May 2023 09:38:50 +0800 Subject: [PATCH 007/159] use getScaleMultiplier --- src/Interpreters/convertFieldToType.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 79251d944c8..1ea9ecd7e00 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -194,13 +194,15 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } else if (which_type.isDateTime64() && which_from_type.isDate()) { - const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); + const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); } else if (which_type.isDateTime64() && which_from_type.isDate32()) { - const auto value = static_cast(type).getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, 1); + const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); + const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { From 93cf34320c12d17554d047497928cac1c177937b Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 30 May 2023 16:10:52 +0200 Subject: [PATCH 008/159] Fixed convertFieldToType case of converting Date and Date32 to DateTime64 Field Also added a unit-test that clarifies implementation --- src/Interpreters/convertFieldToType.cpp | 16 +- .../tests/gtest_convertFieldToType.cpp | 185 ++++++++++++++++++ 2 files changed, 195 insertions(+), 6 deletions(-) create mode 100644 src/Interpreters/tests/gtest_convertFieldToType.cpp diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 1ea9ecd7e00..ff09175f898 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -194,15 +194,19 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } else if (which_type.isDateTime64() && which_from_type.isDate()) { - const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); - const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); + const auto & date_time64_type = static_cast(type); + const auto value = date_time64_type.getTimeZone().fromDayNum(DayNum(src.get())); + return DecimalField( + DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), + date_time64_type.getScale()); } else if (which_type.isDateTime64() && which_from_type.isDate32()) { - const DataTypeDateTime64 & data_type_date_time64 = static_cast(type); - const Int64 value = data_type_date_time64.getTimeZone().fromDayNum(DayNum(src.get())); - return DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, data_type_date_time64.getScaleMultiplier()); + const auto & date_time64_type = static_cast(type); + const auto value = date_time64_type.getTimeZone().fromDayNum(ExtendedDayNum(static_cast(src.get()))); + return DecimalField( + DecimalUtils::decimalFromComponentsWithMultiplier(value, 0, date_time64_type.getScaleMultiplier()), + date_time64_type.getScale()); } else if (type.isValueRepresentedByNumber() && src.getType() != Field::Types::String) { diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp new file mode 100644 index 00000000000..5421c192ac7 --- /dev/null +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -0,0 +1,185 @@ +#include +#include +#include +#include +#include +#include +#include + +#include +#include "base/Decimal.h" +#include "base/types.h" +#include "gtest/gtest.h" + +using namespace DB; + +struct ConvertFieldToTypeTestParams +{ + const char * from_type; // MUST NOT BE NULL + const Field from_value; + const char * to_type; // MUST NOT BE NULL + const std::optional expected_value; +}; + +std::ostream & operator << (std::ostream & ostr, const ConvertFieldToTypeTestParams & params) +{ + return ostr << "{" + << "\n\tfrom_type : " << params.from_type + << "\n\tfrom_value : " << params.from_value + << "\n\tto_type : " << params.to_type + << "\n\texpected : " << (params.expected_value ? *params.expected_value : Field()) + << "\n}"; +} + +class ConvertFieldToTypeTest : public ::testing::TestWithParam +{}; + +TEST_P(ConvertFieldToTypeTest, convert) +{ + const auto & params = GetParam(); + + ASSERT_NE(nullptr, params.from_type); + ASSERT_NE(nullptr, params.to_type); + + const auto & type_factory = DataTypeFactory::instance(); + const auto from_type = type_factory.get(params.from_type); + const auto to_type = type_factory.get(params.to_type); + + if (params.expected_value) + { + const auto result = convertFieldToType(params.from_value, *to_type, from_type.get()); + EXPECT_EQ(*params.expected_value, result); + } + else + { + EXPECT_ANY_THROW(convertFieldToType(params.from_value, *to_type, from_type.get())); + } +} + +// Basically nuber of seconds in a day, works for UTC here +const long long int Day = 24 * 60 * 60; + +// 123 is arbitrary value here + +INSTANTIATE_TEST_SUITE_P( + DateToDateTime64, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + // min value of Date + { + "Date", + Field(0), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(0), 0) + }, + // Max value of Date + { + "Date", + Field(std::numeric_limits::max()), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(std::numeric_limits::max() * Day), 0) + }, + // check that scale is respected + { + "Date", + Field(123), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(123 * Day), 0) + }, + { + "Date", + Field(1), + "DateTime64(1, 'UTC')", + DecimalField(DateTime64(Day * 10), 1) + }, + { + "Date", + Field(123), + "DateTime64(3, 'UTC')", + DecimalField(DateTime64(123 * Day * 1000), 3) + }, + { + "Date", + Field(123), + "DateTime64(6, 'UTC')", + DecimalField(DateTime64(123 * Day * 1'000'000), 6) + }, + }) +); + +INSTANTIATE_TEST_SUITE_P( + Date32ToDateTime64, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + // min value of Date32: 1st Jan 1900 (see DATE_LUT_MIN_YEAR) + { + "Date32", + Field(-25'567), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(-25'567 * Day), 0) + }, + // max value of Date32: 31 Dec 2299 (see DATE_LUT_MAX_YEAR) + { + "Date32", + Field(120'529), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(120'529 * Day), 0) + }, + // check that scale is respected + { + "Date32", + Field(123), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(123 * Day), 0) + }, + { + "Date32", + Field(123), + "DateTime64(1, 'UTC')", + DecimalField(DateTime64(123 * Day * 10), 1) + }, + { + "Date32", + Field(123), + "DateTime64(3, 'UTC')", + DecimalField(DateTime64(123 * Day * 1000), 3) + }, + { + "Date32", + Field(123), + "DateTime64(6, 'UTC')", + DecimalField(DateTime64(123 * Day * 1'000'000), 6) + } + }) + ); + +INSTANTIATE_TEST_SUITE_P( + DateTimeToDateTime64, + ConvertFieldToTypeTest, + ::testing::ValuesIn(std::initializer_list{ + { + "DateTime", + Field(1), + "DateTime64(0, 'UTC')", + DecimalField(DateTime64(1), 0) + }, + { + "DateTime", + Field(1), + "DateTime64(1, 'UTC')", + DecimalField(DateTime64(1'0), 1) + }, + { + "DateTime", + Field(123), + "DateTime64(3, 'UTC')", + DecimalField(DateTime64(123'000), 3) + }, + { + "DateTime", + Field(123), + "DateTime64(6, 'UTC')", + DecimalField(DateTime64(123'000'000), 6) + }, + }) +); From a4cb82127dfe488fb1e1ad90aebaccf469ad742e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 13:54:33 +0200 Subject: [PATCH 009/159] Analyzer: WIP on distributed queries --- src/Analyzer/ColumnNode.cpp | 5 ++++- src/Analyzer/TableNode.cpp | 2 ++ src/Planner/PlannerContext.cpp | 14 +++++++------- src/Planner/PlannerJoinTree.cpp | 3 ++- src/Planner/Utils.cpp | 2 +- src/Storages/StorageDistributed.cpp | 6 ++++-- 6 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index a9d47f8287d..f020040ea78 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -117,7 +117,10 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const else { const auto & table_storage_id = table_node->getStorageID(); - column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; + if (table_storage_id.hasDatabase()) + column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; + else + column_identifier_parts = { table_storage_id.getTableName() }; } } } diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index c86cbcd5a80..17d12bd6afa 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -91,6 +91,8 @@ ASTPtr TableNode::toASTImpl(const ConvertToASTOptions & /* options */) const if (!temporary_table_name.empty()) return std::make_shared(temporary_table_name); + if (!storage_id.hasDatabase()) + return std::make_shared(storage_id.getTableName()); return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); } diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 346cc6d2080..7ab8180eb9c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -19,17 +19,17 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Quer return createColumnIdentifier(column_node_typed.getColumn(), column_source_node); } -const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & column_source_node) +const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const NameAndTypePair & column, const QueryTreeNodePtr & /*column_source_node*/) { std::string column_identifier; - if (column_source_node->hasAlias()) - column_identifier += column_source_node->getAlias(); - else if (const auto * table_source_node = column_source_node->as()) - column_identifier += table_source_node->getStorageID().getFullNameNotQuoted(); + // if (column_source_node->hasAlias()) + // column_identifier += column_source_node->getAlias(); + // else if (const auto * table_source_node = column_source_node->as()) + // column_identifier += table_source_node->getStorageID().getFullNameNotQuoted(); - if (!column_identifier.empty()) - column_identifier += '.'; + // if (!column_identifier.empty()) + // column_identifier += '.'; column_identifier += column.name; column_identifier += '_' + std::to_string(column_identifiers.size()); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4f091f73187..4a1708f96d3 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -106,7 +106,8 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, storage_id.getFullTableName()); } - query_context->checkAccess(AccessType::SELECT, storage_id, column_names); + if (storage_id.hasDatabase()) + query_context->checkAccess(AccessType::SELECT, storage_id, column_names); } NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot) diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 5c61b2fc2c7..cd4fb9182e9 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -107,7 +107,7 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers, SelectUnionMode ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) { auto & query_node_typed = query_node->as(); - auto result_ast = query_node_typed.toAST(); + auto result_ast = query_node_typed.toAST({ .fully_qualified_identifiers = false }); while (true) { diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bcf6f68d00d..e0bb14c62fd 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -943,7 +943,9 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, } else { - auto resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); + auto resolved_remote_storage_id = remote_storage_id; + if (remote_storage_id.hasDatabase()) + resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); auto table_node = std::make_shared(std::move(storage), query_context); @@ -1059,7 +1061,7 @@ void StorageDistributed::read( remote_table_function_ptr); query_ast = queryNodeToSelectQuery(query_tree_distributed); - header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_ast, local_context, SelectQueryOptions(processed_stage).analyze()); + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); } else { From 70c1b1de57252ace05fe69b3a791b3fe4f025273 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 15:43:30 +0000 Subject: [PATCH 010/159] Remove code --- src/Planner/PlannerContext.cpp | 8 -- ...ns_optimize_read_in_window_order.reference | 10 +- .../02227_union_match_by_name.reference | 4 +- .../0_stateless/02303_query_kind.reference | 16 +-- .../02381_join_dup_columns_in_plan.reference | 44 +++--- .../0_stateless/02421_explain_subquery.sql | 2 +- .../02451_order_by_monotonic.reference | 32 ++--- .../02481_aggregation_in_order_plan.reference | 2 +- .../02514_analyzer_drop_join_on.reference | 134 +++++++++--------- ...dicate_push_down_filled_join_fix.reference | 30 ++-- ...n_merge_tree_prewhere_row_policy.reference | 8 +- 11 files changed, 141 insertions(+), 149 deletions(-) diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 7ab8180eb9c..a788a6cbc3c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -23,14 +23,6 @@ const ColumnIdentifier & GlobalPlannerContext::createColumnIdentifier(const Name { std::string column_identifier; - // if (column_source_node->hasAlias()) - // column_identifier += column_source_node->getAlias(); - // else if (const auto * table_source_node = column_source_node->as()) - // column_identifier += table_source_node->getStorageID().getFullNameNotQuoted(); - - // if (!column_identifier.empty()) - // column_identifier += '.'; - column_identifier += column.name; column_identifier += '_' + std::to_string(column_identifiers.size()); diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference index f23cf03913b..8a33df9fad2 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.reference @@ -7,19 +7,19 @@ Partial sorting plan Prefix sort description: n ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n.n_0 ASC - Result sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n.x_1 ASC + Prefix sort description: n_0 ASC + Result sort description: n_0 ASC, x_1 ASC No sorting plan optimize_read_in_window_order=0 Sort description: n ASC, x ASC optimize_read_in_window_order=0, allow_experimental_analyzer=1 - Sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.x_1 ASC + Sort description: n_0 ASC, x_1 ASC optimize_read_in_window_order=1 Prefix sort description: n ASC, x ASC Result sort description: n ASC, x ASC optimize_read_in_window_order=1, allow_experimental_analyzer=1 - Prefix sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.x_1 ASC - Result sort description: default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.n_0 ASC, default.test_01655_plan_optimizations_optimize_read_in_window_order_n_x.x_1 ASC + Prefix sort description: n_0 ASC, x_1 ASC + Result sort description: n_0 ASC, x_1 ASC Complex ORDER BY optimize_read_in_window_order=0 3 3 1 diff --git a/tests/queries/0_stateless/02227_union_match_by_name.reference b/tests/queries/0_stateless/02227_union_match_by_name.reference index e51ea983f7f..685b3c83b05 100644 --- a/tests/queries/0_stateless/02227_union_match_by_name.reference +++ b/tests/queries/0_stateless/02227_union_match_by_name.reference @@ -30,7 +30,7 @@ Header: avgWeighted(x, y) Nullable(Float64) 255_UInt8 UInt8 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: system.one.dummy_0 UInt8 + Header: dummy_0 UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 Expression (Conversion before UNION) @@ -46,7 +46,7 @@ Header: avgWeighted(x, y) Nullable(Float64) NULL_Nullable(Nothing) Nullable(Nothing) 1_UInt8 UInt8 Expression (Change column names to column identifiers) - Header: system.one.dummy_0 UInt8 + Header: dummy_0 UInt8 ReadFromStorage (SystemOne) Header: dummy UInt8 SELECT avgWeighted(x, y) FROM (SELECT NULL, 255 AS x, 1 AS y UNION ALL SELECT y, NULL AS x, 1 AS y); diff --git a/tests/queries/0_stateless/02303_query_kind.reference b/tests/queries/0_stateless/02303_query_kind.reference index 5af8c2b743f..8d119fb22b2 100644 --- a/tests/queries/0_stateless/02303_query_kind.reference +++ b/tests/queries/0_stateless/02303_query_kind.reference @@ -2,35 +2,35 @@ clickhouse-client --allow_experimental_analyzer=1 --query_kind secondary_query - Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind secondary_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-client --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 clickhouse-local --allow_experimental_analyzer=1 --query_kind initial_query -q explain plan header=1 select toString(dummy) as dummy from system.one group by dummy Expression ((Project names + Projection)) Header: dummy String Aggregating - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String Expression ((Before GROUP BY + Change column names to column identifiers)) - Header: toString(system.one.dummy_0) String + Header: toString(dummy_0) String ReadFromStorage (SystemOne) Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index 31a37862663..dd5c9d4616e 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -2,51 +2,51 @@ Expression Header: key String value String Join - Header: s1.key_0 String - s2.value_1 String + Header: key_0 String + value_1 String Expression - Header: s1.key_0 String + Header: key_0 String ReadFromStorage Header: dummy UInt8 Union - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 Expression Header: key String value String Join - Header: s1.key_0 String - s2.key_2 String - s2.value_1 String + Header: key_0 String + key_2 String + value_1 String Sorting - Header: s1.key_0 String + Header: key_0 String Expression - Header: s1.key_0 String + Header: key_0 String ReadFromStorage Header: dummy UInt8 Sorting - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String Union - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 Expression - Header: s2.key_2 String - s2.value_1 String + Header: key_2 String + value_1 String ReadFromStorage Header: dummy UInt8 diff --git a/tests/queries/0_stateless/02421_explain_subquery.sql b/tests/queries/0_stateless/02421_explain_subquery.sql index 32631b54d0c..4b970f81219 100644 --- a/tests/queries/0_stateless/02421_explain_subquery.sql +++ b/tests/queries/0_stateless/02421_explain_subquery.sql @@ -34,7 +34,7 @@ DROP TABLE t1; SET allow_experimental_analyzer = 1; -SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: system.numbers.number__ UInt64%'; +SELECT count() > 3 FROM (EXPLAIN PIPELINE header = 1 SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain LIKE '%Header: number__ UInt64%'; SELECT count() > 0 FROM (EXPLAIN PLAN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN SELECT * FROM system.numbers ORDER BY number DESC) WHERE explain ILIKE '%Sort%'; SELECT count() > 0 FROM (EXPLAIN CURRENT TRANSACTION); diff --git a/tests/queries/0_stateless/02451_order_by_monotonic.reference b/tests/queries/0_stateless/02451_order_by_monotonic.reference index f9f0ef38be1..05f20a9bad8 100644 --- a/tests/queries/0_stateless/02451_order_by_monotonic.reference +++ b/tests/queries/0_stateless/02451_order_by_monotonic.reference @@ -4,19 +4,19 @@ 2022-09-09 12:00:00 0x 2022-09-09 12:00:00 1 2022-09-09 12:00:00 1x - Prefix sort description: toStartOfMinute(test.t_0) ASC - Result sort description: toStartOfMinute(test.t_0) ASC, test.c1_1 ASC - Prefix sort description: toStartOfMinute(test.t_0) ASC - Result sort description: toStartOfMinute(test.t_0) ASC - Prefix sort description: negate(test.a_0) ASC - Result sort description: negate(test.a_0) ASC - Prefix sort description: negate(test.a_0) ASC, negate(test.b_1) ASC - Result sort description: negate(test.a_0) ASC, negate(test.b_1) ASC - Prefix sort description: test.a_0 DESC, negate(test.b_1) ASC - Result sort description: test.a_0 DESC, negate(test.b_1) ASC - Prefix sort description: negate(test.a_0) ASC, test.b_1 DESC - Result sort description: negate(test.a_0) ASC, test.b_1 DESC - Prefix sort description: negate(test.a_0) ASC - Result sort description: negate(test.a_0) ASC, test.b_1 ASC - Prefix sort description: test.a_0 ASC - Result sort description: test.a_0 ASC, negate(test.b_1) ASC + Prefix sort description: toStartOfMinute(t_0) ASC + Result sort description: toStartOfMinute(t_0) ASC, c1_1 ASC + Prefix sort description: toStartOfMinute(t_0) ASC + Result sort description: toStartOfMinute(t_0) ASC + Prefix sort description: negate(a_0) ASC + Result sort description: negate(a_0) ASC + Prefix sort description: negate(a_0) ASC, negate(b_1) ASC + Result sort description: negate(a_0) ASC, negate(b_1) ASC + Prefix sort description: a_0 DESC, negate(b_1) ASC + Result sort description: a_0 DESC, negate(b_1) ASC + Prefix sort description: negate(a_0) ASC, b_1 DESC + Result sort description: negate(a_0) ASC, b_1 DESC + Prefix sort description: negate(a_0) ASC + Result sort description: negate(a_0) ASC, b_1 ASC + Prefix sort description: a_0 ASC + Result sort description: a_0 ASC, negate(b_1) ASC diff --git a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference index bb4eb4ddd75..b11f3e3a1d3 100644 --- a/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference +++ b/tests/queries/0_stateless/02481_aggregation_in_order_plan.reference @@ -6,5 +6,5 @@ Order: a ASC, c ASC ReadFromMergeTree (default.tab) Aggregating - Order: default.tab.a_0 ASC, default.tab.c_2 ASC + Order: a_0 ASC, c_2 ASC ReadFromMergeTree (default.tab) diff --git a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference index 8b4cafc3260..0037ab85c07 100644 --- a/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference +++ b/tests/queries/0_stateless/02514_analyzer_drop_join_on.reference @@ -6,43 +6,43 @@ SELECT count() FROM a JOIN b ON b.b1 = a.a1 JOIN c ON c.c1 = b.b1 JOIN d ON d.d1 Expression ((Project names + Projection)) Header: count() UInt64 Aggregating - Header: default.a.a2_4 String + Header: a2_4 String count() UInt64 Expression ((Before GROUP BY + DROP unused columns after JOIN)) - Header: default.a.a2_4 String + Header: a2_4 String Join (JOIN FillRightFirst) - Header: default.a.a2_4 String - default.c.c1_2 UInt64 + Header: a2_4 String + c1_2 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: default.a.a2_4 String - default.c.c1_2 UInt64 + Header: a2_4 String + c1_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_4 String - default.b.b1_0 UInt64 - default.c.c1_2 UInt64 + Header: a2_4 String + b1_0 UInt64 + c1_2 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: default.a.a2_4 String - default.b.b1_0 UInt64 + Header: a2_4 String + b1_0 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a1_1 UInt64 - default.a.a2_4 String - default.b.b1_0 UInt64 + Header: a1_1 UInt64 + a2_4 String + b1_0 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.a.a1_1 UInt64 - default.a.a2_4 String + Header: a1_1 UInt64 + a2_4 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.b.b1_0 UInt64 + Header: b1_0 UInt64 ReadFromMemoryStorage Header: b1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.c.c1_2 UInt64 + Header: c1_2 UInt64 ReadFromMemoryStorage Header: c1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.d.d1_3 UInt64 + Header: d1_3 UInt64 ReadFromMemoryStorage Header: d1 UInt64 EXPLAIN PLAN header = 1 @@ -52,38 +52,38 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN))) Header: a2 String d2 String Join (JOIN FillRightFirst) - Header: default.a.a2_0 String - default.a.k_2 UInt64 - default.d.d2_1 String + Header: a2_0 String + k_2 UInt64 + d2_1 String Expression (DROP unused columns after JOIN) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Expression (DROP unused columns after JOIN) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 Expression (Change column names to column identifiers) - Header: default.a.a2_0 String - default.a.k_2 UInt64 + Header: a2_0 String + k_2 UInt64 ReadFromMemoryStorage Header: a2 String k UInt64 Expression (Change column names to column identifiers) - Header: default.b.k_3 UInt64 + Header: k_3 UInt64 ReadFromMemoryStorage Header: k UInt64 Expression (Change column names to column identifiers) - Header: default.c.k_4 UInt64 + Header: k_4 UInt64 ReadFromMemoryStorage Header: k UInt64 Expression (Change column names to column identifiers) - Header: default.d.d2_1 String - default.d.k_5 UInt64 + Header: d2_1 String + k_5 UInt64 ReadFromMemoryStorage Header: d2 String k UInt64 @@ -97,55 +97,55 @@ WHERE c.c2 != '' ORDER BY a.a2 Expression (Project names) Header: bx String Sorting (Sorting for ORDER BY) - Header: default.a.a2_6 String - b.bx_0 String + Header: a2_6 String + bx_0 String Expression ((Before ORDER BY + (Projection + ))) - Header: default.a.a2_6 String - b.bx_0 String + Header: a2_6 String + bx_0 String Join (JOIN FillRightFirst) - Header: default.a.a2_6 String - b.bx_0 String - default.c.c2_5 String - default.c.c1_3 UInt64 + Header: a2_6 String + bx_0 String + c2_5 String + c1_3 UInt64 Filter (( + (JOIN actions + DROP unused columns after JOIN))) - Header: default.a.a2_6 String - b.bx_0 String - default.c.c2_5 String - default.c.c1_3 UInt64 + Header: a2_6 String + bx_0 String + c2_5 String + c1_3 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a2_6 String - b.bx_0 String - b.b1_1 UInt64 - default.c.c2_5 String - default.c.c1_3 UInt64 + Header: a2_6 String + bx_0 String + b1_1 UInt64 + c2_5 String + c1_3 UInt64 Expression ((JOIN actions + DROP unused columns after JOIN)) - Header: default.a.a2_6 String - b.bx_0 String - b.b1_1 UInt64 + Header: a2_6 String + bx_0 String + b1_1 UInt64 Join (JOIN FillRightFirst) - Header: default.a.a1_2 UInt64 - default.a.a2_6 String - b.bx_0 String - b.b1_1 UInt64 + Header: a1_2 UInt64 + a2_6 String + bx_0 String + b1_1 UInt64 Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.a.a1_2 UInt64 - default.a.a2_6 String + Header: a1_2 UInt64 + a2_6 String ReadFromMemoryStorage Header: a1 UInt64 a2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: b.b1_1 UInt64 - b.bx_0 String + Header: b1_1 UInt64 + bx_0 String ReadFromMemoryStorage Header: b1 UInt64 b2 String Expression ((JOIN actions + Change column names to column identifiers)) - Header: default.c.c1_3 UInt64 - default.c.c2_5 String + Header: c1_3 UInt64 + c2_5 String ReadFromMemoryStorage Header: c1 UInt64 c2 String Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) - Header: d.d1_4 UInt64 + Header: d1_4 UInt64 ReadFromStorage (SystemNumbers) Header: number UInt64 diff --git a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference index ecdb62c5cb5..986ecffcdf8 100644 --- a/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference +++ b/tests/queries/0_stateless/02675_predicate_push_down_filled_join_fix.reference @@ -2,27 +2,27 @@ Expression ((Project names + (Projection + ))) Header: t1.id UInt64 t1.value String t2.value String -Actions: INPUT : 0 -> t1.id_0 UInt64 : 0 - INPUT : 1 -> t1.value_1 String : 1 - INPUT : 2 -> t2.value_2 String : 2 - ALIAS t1.id_0 :: 0 -> t1.id UInt64 : 3 - ALIAS t1.value_1 :: 1 -> t1.value String : 0 - ALIAS t2.value_2 :: 2 -> t2.value String : 1 +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 String : 2 + ALIAS id_0 :: 0 -> t1.id UInt64 : 3 + ALIAS value_1 :: 1 -> t1.value String : 0 + ALIAS value_2 :: 2 -> t2.value String : 1 Positions: 3 0 1 FilledJoin (Filled JOIN) - Header: t1.id_0 UInt64 - t1.value_1 String - t2.value_2 String + Header: id_0 UInt64 + value_1 String + value_2 String Filter (( + (JOIN actions + Change column names to column identifiers))) - Header: t1.id_0 UInt64 - t1.value_1 String - Filter column: equals(t1.id_0, 0_UInt8) (removed) + Header: id_0 UInt64 + value_1 String + Filter column: equals(id_0, 0_UInt8) (removed) Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 COLUMN Const(UInt8) -> 0_UInt8 UInt8 : 2 - ALIAS id :: 0 -> t1.id_0 UInt64 : 3 - ALIAS value :: 1 -> t1.value_1 String : 0 - FUNCTION equals(t1.id_0 : 3, 0_UInt8 :: 2) -> equals(t1.id_0, 0_UInt8) UInt8 : 1 + ALIAS id :: 0 -> id_0 UInt64 : 3 + ALIAS value :: 1 -> value_1 String : 0 + FUNCTION equals(id_0 : 3, 0_UInt8 :: 2) -> equals(id_0, 0_UInt8) UInt8 : 1 Positions: 1 3 0 ReadFromMergeTree (default.test_table) Header: id UInt64 diff --git a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference index 2fe98ea1682..cc16a1fce02 100644 --- a/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference +++ b/tests/queries/0_stateless/02679_explain_merge_tree_prewhere_row_policy.reference @@ -29,10 +29,10 @@ Header: id UInt64 value String Actions: INPUT : 0 -> id UInt64 : 0 INPUT : 1 -> value String : 1 - ALIAS id :: 0 -> default.test_table.id_0 UInt64 : 2 - ALIAS value :: 1 -> default.test_table.value_1 String : 0 - ALIAS default.test_table.id_0 :: 2 -> id UInt64 : 1 - ALIAS default.test_table.value_1 :: 0 -> value String : 2 + ALIAS id :: 0 -> id_0 UInt64 : 2 + ALIAS value :: 1 -> value_1 String : 0 + ALIAS id_0 :: 2 -> id UInt64 : 1 + ALIAS value_1 :: 0 -> value String : 2 Positions: 1 2 ReadFromMergeTree (default.test_table) Header: id UInt64 From 2c878581bf0642b7e7d8b3c63ac483f6f102639d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 22 May 2023 17:20:53 +0000 Subject: [PATCH 011/159] Update 02377_optimize_sorting_by_input_stream_properties_explain reference --- ..._input_stream_properties_explain.reference | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index 69c325c21a9..5c9e39805b7 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -8,7 +8,7 @@ Sorting (None) -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_0 ASC +Sorting (Global): a_0 ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort @@ -36,8 +36,8 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_0 ASC -Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Global): a_0 ASC +Sorting (Stream): a_0 ASC Sorting (Stream): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) @@ -48,8 +48,8 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Global): plus(a_0, 1_UInt8) ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 @@ -61,7 +61,7 @@ Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(default.optimize_sorting.a_0, 1_UInt8) ASC +Sorting (Global): plus(a_0, 1_UInt8) ASC Sorting (None) Sorting (Chunk): a ASC -- FilterStep preserves sort mode @@ -71,7 +71,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -79,7 +79,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC @@ -87,7 +87,7 @@ Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC -Sorting (Chunk): default.optimize_sorting.a_0 ASC +Sorting (Chunk): a_0 ASC Sorting (Chunk): a ASC -- FilterStep breaks sort mode -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 @@ -122,8 +122,8 @@ Sorting (Sorting for ORDER BY) Sorting (Global): a_0 ASC Sorting (None) Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_2 ASC -Sorting (Stream): default.optimize_sorting.a_2 ASC +Sorting (Global): a_2 ASC +Sorting (Stream): a_2 ASC Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) @@ -152,10 +152,10 @@ Sorting (Chunk): a ASC Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a_0, 1_UInt8) ASC -Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC +Sorting (Global): plus(a_3, 1_UInt8) ASC Sorting (Sorting for ORDER BY) -Sorting (Global): plus(default.optimize_sorting.a_3, 1_UInt8) ASC -Sorting (Chunk): default.optimize_sorting.a_3 ASC +Sorting (Global): plus(a_3, 1_UInt8) ASC +Sorting (Chunk): a_3 ASC Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query -- QUERY: set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a @@ -167,6 +167,6 @@ Sorting (Stream): a ASC -- QUERY (analyzer): set optimize_read_in_order=1;set max_threads=3;set query_plan_remove_redundant_sorting=0;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) -Sorting (Global): default.optimize_sorting.a_0 ASC -Sorting (Stream): default.optimize_sorting.a_0 ASC +Sorting (Global): a_0 ASC +Sorting (Stream): a_0 ASC Sorting (Stream): a ASC From d79bd5694ad6345e41aa640afb4d839c46da716d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 24 May 2023 00:05:27 +0200 Subject: [PATCH 012/159] Fix LambdaNode::cloneImpl --- src/Analyzer/LambdaNode.cpp | 5 +++-- src/Analyzer/LambdaNode.h | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/LambdaNode.cpp b/src/Analyzer/LambdaNode.cpp index 0d15c4d42e6..4be4d69c190 100644 --- a/src/Analyzer/LambdaNode.cpp +++ b/src/Analyzer/LambdaNode.cpp @@ -10,9 +10,10 @@ namespace DB { -LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_) +LambdaNode::LambdaNode(Names argument_names_, QueryTreeNodePtr expression_, DataTypePtr result_type_) : IQueryTreeNode(children_size) , argument_names(std::move(argument_names_)) + , result_type(std::move(result_type_)) { auto arguments_list_node = std::make_shared(); auto & nodes = arguments_list_node->getNodes(); @@ -63,7 +64,7 @@ void LambdaNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr LambdaNode::cloneImpl() const { - return std::make_shared(argument_names, getExpression()); + return std::make_shared(argument_names, getExpression(), result_type); } ASTPtr LambdaNode::toASTImpl(const ConvertToASTOptions & options) const diff --git a/src/Analyzer/LambdaNode.h b/src/Analyzer/LambdaNode.h index 355ed77cc6a..ea44a7e8187 100644 --- a/src/Analyzer/LambdaNode.h +++ b/src/Analyzer/LambdaNode.h @@ -35,7 +35,7 @@ class LambdaNode final : public IQueryTreeNode { public: /// Initialize lambda with argument names and lambda body expression - explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_); + explicit LambdaNode(Names argument_names_, QueryTreeNodePtr expression_, DataTypePtr result_type_ = {}); /// Get argument names const Names & getArgumentNames() const From b86516131bef352b114e67c17b77706496bb0fd9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 24 May 2023 16:22:58 +0000 Subject: [PATCH 013/159] Attempt to fix global JOINs and INs --- src/Storages/StorageDistributed.cpp | 29 +++++++++++++++++++++++++---- 1 file changed, 25 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index e0bb14c62fd..05f9821f6cb 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,6 +30,8 @@ #include #include #include +#include "Analyzer/IQueryTreeNode.h" +#include "Analyzer/MatcherNode.h" #include #include @@ -1003,7 +1005,17 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); - replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); + + auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); + in_second_argument_query_node->setIsSubquery(true); + in_second_argument_query_node->getProjectionNode() = std::make_shared(); + in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; + in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); + + QueryAnalysisPass query_analysis_pass; + query_analysis_pass.run(in_second_argument_query_node, query_context); + + replacement_map.emplace(join_right_table_expression.get(), std::move(in_second_argument_query_node)); continue; } else if (auto * in_function_node = global_in_or_join_node.query_node->as()) @@ -1016,7 +1028,17 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); - in_function_subquery_node = std::move(temporary_table_expression_node); + + auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); + in_second_argument_query_node->setIsSubquery(true); + in_second_argument_query_node->getProjectionNode() = std::make_shared(); + in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; + in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); + + QueryAnalysisPass query_analysis_pass; + query_analysis_pass.run(in_second_argument_query_node, query_context); + + in_function_subquery_node = std::move(in_second_argument_query_node); } else { @@ -1059,9 +1081,8 @@ void StorageDistributed::read( storage_snapshot, remote_storage_id, remote_table_function_ptr); - - query_ast = queryNodeToSelectQuery(query_tree_distributed); header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); + query_ast = queryNodeToSelectQuery(query_tree_distributed); } else { From 85e5ed79e5b60f00df2cf7d8c41b249485c02547 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 25 May 2023 15:58:08 +0000 Subject: [PATCH 014/159] Fix distributed JOINs --- src/Storages/StorageDistributed.cpp | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 05f9821f6cb..98c92541f67 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1006,16 +1006,7 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, global_in_or_join_node.subquery_depth); temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); - auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); - in_second_argument_query_node->setIsSubquery(true); - in_second_argument_query_node->getProjectionNode() = std::make_shared(); - in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; - in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); - - QueryAnalysisPass query_analysis_pass; - query_analysis_pass.run(in_second_argument_query_node, query_context); - - replacement_map.emplace(join_right_table_expression.get(), std::move(in_second_argument_query_node)); + replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); continue; } else if (auto * in_function_node = global_in_or_join_node.query_node->as()) From bc7b7f2cd2e1c56aacecd2855feb37bac10590c6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 25 May 2023 16:00:13 +0000 Subject: [PATCH 015/159] Always add source alias in ColumnNode::toASTImpl --- src/Analyzer/ColumnNode.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index f020040ea78..76f5697afb0 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -96,7 +96,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const std::vector column_identifier_parts; auto column_source = getColumnSourceOrNull(); - if (column_source && options.fully_qualified_identifiers) + if (column_source) { auto node_type = column_source->getNodeType(); if (node_type == QueryTreeNodeType::TABLE || @@ -108,7 +108,8 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const { column_identifier_parts = {column_source->getAlias()}; } - else if (auto * table_node = column_source->as()) + else if (auto * table_node = column_source->as(); + table_node && options.fully_qualified_identifiers) { if (!table_node->getTemporaryTableName().empty()) { From c6dcb69b853528aacda070d00b3a873179470f82 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 30 May 2023 14:33:35 +0000 Subject: [PATCH 016/159] Fix GLOBAL IN --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 9 +++--- src/Planner/CollectSets.cpp | 3 +- src/Planner/Planner.cpp | 35 +++++++++++++++++++++-- src/Planner/PlannerContext.cpp | 3 +- src/Storages/StorageDistributed.cpp | 14 ++------- 5 files changed, 45 insertions(+), 19 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b2bfa648435..c454ad9f84f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4767,13 +4767,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto * table_node = in_second_argument->as(); auto * table_function_node = in_second_argument->as(); - if (table_node && dynamic_cast(table_node->getStorage().get()) != nullptr) + if (table_node) { - /// If table is already prepared set, we do not replace it with subquery + /// If table is already prepared set, we do not replace it with subquery. + /// If table is not a StorageSet, we'll create plan to build set in the Planner. } - else if (table_node || table_function_node) + else if (table_function_node) { - const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); + const auto & storage_snapshot = table_function_node->getStorageSnapshot(); auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); size_t columns_to_select_size = columns_to_select.size(); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 02069aad292..eb2b02c7ccb 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -67,7 +67,8 @@ public: planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || - in_second_argument_node_type == QueryTreeNodeType::UNION) + in_second_argument_node_type == QueryTreeNodeType::UNION || + in_second_argument_node_type == QueryTreeNodeType::TABLE) { planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 7292f73e21c..5abf3ec5a80 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -43,6 +43,7 @@ #include #include +#include #include #include #include @@ -909,12 +910,42 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, if (!planner_set) continue; - if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) + auto subquery_to_execute = planner_set->getSubqueryNode(); + + if (planner_set->getSet().isCreated() || !subquery_to_execute) continue; + if (auto * table_node = subquery_to_execute->as()) + { + auto storage_snapshot = table_node->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + + size_t columns_to_select_size = columns_to_select.size(); + + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); + + for (auto & column : columns_to_select) + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); + projection_columns.emplace_back(column.name, column.type); + } + + auto subquery_for_table = std::make_shared(Context::createCopy(planner_context->getQueryContext())); + subquery_for_table->setIsSubquery(true); + subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); + subquery_for_table->getJoinTree() = std::move(subquery_to_execute); + subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); + + subquery_to_execute = std::move(subquery_for_table); + } + auto subquery_options = select_query_options.subquery(); Planner subquery_planner( - planner_set->getSubqueryNode(), + subquery_to_execute, subquery_options, planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index a788a6cbc3c..708dab04d02 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -129,7 +129,8 @@ void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) auto node_type = subquery_node->getNodeType(); if (node_type != QueryTreeNodeType::QUERY && - node_type != QueryTreeNodeType::UNION) + node_type != QueryTreeNodeType::UNION && + node_type != QueryTreeNodeType::TABLE) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid node for set table expression. Expected query or union. Actual {}", subquery_node->formatASTForErrorMessage()); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 98c92541f67..f1fb4bb0c65 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -849,7 +849,7 @@ private: /** Execute subquery node and put result in mutable context temporary table. * Returns table node that is initialized with temporary table storage. */ -QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, +TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, ContextMutablePtr & mutable_context, size_t subquery_depth) { @@ -1019,17 +1019,9 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); + temporary_table_expression_node->setAlias(temporary_table_expression_node->getTemporaryTableName()); - auto in_second_argument_query_node = std::make_shared(Context::createCopy(query_context)); - in_second_argument_query_node->setIsSubquery(true); - in_second_argument_query_node->getProjectionNode() = std::make_shared(); - in_second_argument_query_node->getProjection().getNodes() = { std::make_shared() }; - in_second_argument_query_node->getJoinTree() = std::move(temporary_table_expression_node); - - QueryAnalysisPass query_analysis_pass; - query_analysis_pass.run(in_second_argument_query_node, query_context); - - in_function_subquery_node = std::move(in_second_argument_query_node); + in_function_subquery_node = std::move(temporary_table_expression_node); } else { From eb7ae91d0144a895ef53c862c79d930fe2cbdbab Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 May 2023 15:00:11 +0000 Subject: [PATCH 017/159] Do not add alias to a temporary table --- src/Storages/StorageDistributed.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f1fb4bb0c65..43b1333413e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1019,7 +1019,6 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto temporary_table_expression_node = executeSubqueryNode(in_function_subquery_node, planner_context->getMutableQueryContext(), global_in_or_join_node.subquery_depth); - temporary_table_expression_node->setAlias(temporary_table_expression_node->getTemporaryTableName()); in_function_subquery_node = std::move(temporary_table_expression_node); } From 99b35eca0789cd9e3a055697486532bc1d4403d1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 31 May 2023 15:05:48 +0000 Subject: [PATCH 018/159] Update reference files --- .../01561_clickhouse_client_stage.reference | 2 +- .../0_stateless/01591_window_functions.reference | 14 +++++++------- .../02048_clickhouse_local_stage.reference | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference index 00e0f4ddb2e..8a34751b071 100644 --- a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"system.one.dummy_0" +"dummy_0" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index b981a46b4fd..8939ea1111d 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -917,9 +917,9 @@ from ; Expression ((Project names + Projection)) Window (Window step for window \'\') - Window (Window step for window \'PARTITION BY t.p_0\') - Window (Window step for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\') - Sorting (Sorting for window \'PARTITION BY t.p_0 ORDER BY t.o_1 ASC\') + Window (Window step for window \'PARTITION BY p_0\') + Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') + Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromStorage (SystemNumbers) explain select @@ -930,11 +930,11 @@ from from numbers(16)) t ; Expression ((Project names + Projection)) - Window (Window step for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\') - Sorting (Sorting for window \'ORDER BY t.o_0 ASC, t.number_1 ASC\') - Window (Window step for window \'ORDER BY t.number_1 ASC\') + Window (Window step for window \'ORDER BY o_0 ASC, number_1 ASC\') + Sorting (Sorting for window \'ORDER BY o_0 ASC, number_1 ASC\') + Window (Window step for window \'ORDER BY number_1 ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part]) - Sorting (Sorting for window \'ORDER BY t.number_1 ASC\') + Sorting (Sorting for window \'ORDER BY number_1 ASC\') Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) ReadFromStorage (SystemNumbers) -- A test case for the sort comparator found by fuzzer. diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference index 00e0f4ddb2e..8a34751b071 100644 --- a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -2,7 +2,7 @@ execute: --allow_experimental_analyzer=1 "foo" 1 execute: --allow_experimental_analyzer=1 --stage fetch_columns -"system.one.dummy_0" +"dummy_0" 0 execute: --allow_experimental_analyzer=1 --stage with_mergeable_state "1_UInt8" From 480db8622df10993a2df31e24255f96f58ef0094 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 1 Jun 2023 15:21:55 +0000 Subject: [PATCH 019/159] Always add table name while converting ColumnNode to AST --- src/Analyzer/ColumnNode.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index 76f5697afb0..dd41522ac7d 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -108,8 +108,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const { column_identifier_parts = {column_source->getAlias()}; } - else if (auto * table_node = column_source->as(); - table_node && options.fully_qualified_identifiers) + else if (auto * table_node = column_source->as()) { if (!table_node->getTemporaryTableName().empty()) { @@ -118,7 +117,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const else { const auto & table_storage_id = table_node->getStorageID(); - if (table_storage_id.hasDatabase()) + if (table_storage_id.hasDatabase() && options.fully_qualified_identifiers) column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; else column_identifier_parts = { table_storage_id.getTableName() }; From 79c14c89ee39bea6017e41e375f9b6a59e3e7899 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Jun 2023 12:53:25 +0200 Subject: [PATCH 020/159] Add some assertions --- src/Interpreters/Cache/FileCache.cpp | 6 +++--- src/Interpreters/Cache/FileSegment.cpp | 18 +++++++++++++----- src/Interpreters/Cache/Metadata.cpp | 21 ++++++++++++--------- src/Interpreters/Cache/Metadata.h | 4 ++-- 4 files changed, 30 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 1908a4ce895..163e1b71ab9 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -48,12 +48,12 @@ const String & FileCache::getBasePath() const String FileCache::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const { - return metadata.getPathInLocalCache(key, offset, segment_kind); + return metadata.getPathForFileSegment(key, offset, segment_kind); } String FileCache::getPathInLocalCache(const Key & key) const { - return metadata.getPathInLocalCache(key); + return metadata.getPathForKey(key); } void FileCache::assertInitialized() const @@ -1019,7 +1019,7 @@ std::vector FileCache::tryGetCachePaths(const Key & key) for (const auto & [offset, file_segment_metadata] : *locked_key->getKeyMetadata()) { if (file_segment_metadata->file_segment->state() == FileSegment::State::DOWNLOADED) - cache_paths.push_back(metadata.getPathInLocalCache(key, offset, file_segment_metadata->file_segment->getKind())); + cache_paths.push_back(metadata.getPathForFileSegment(key, offset, file_segment_metadata->file_segment->getKind())); } return cache_paths; } diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 60228573666..fb0ba0eba14 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -313,6 +313,13 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); + auto file_segment_path = getPathInLocalCache(); + if (offset == range().left && fs::exists(file_segment_path)) + { + fs::remove(file_segment_path); + chassert(false); + } + { auto lock = segment_guard.lock(); @@ -365,7 +372,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) downloaded_size += size; - chassert(std::filesystem::file_size(getPathInLocalCache()) == downloaded_size); + chassert(std::filesystem::file_size(file_segment_path) == downloaded_size); } catch (ErrnoException & e) { @@ -375,7 +382,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) int code = e.getErrno(); if (code == /* No space left on device */28 || code == /* Quota exceeded */122) { - const auto file_size = fs::file_size(getPathInLocalCache()); + const auto file_size = fs::file_size(file_segment_path); chassert(downloaded_size <= file_size); chassert(reserved_size >= file_size); if (downloaded_size != file_size) @@ -520,8 +527,8 @@ void FileSegment::setDownloadedUnlocked(const FileSegmentGuard::Lock &) remote_file_reader.reset(); } - chassert(getDownloadedSize(false) > 0); - chassert(fs::file_size(getPathInLocalCache()) > 0); + chassert(downloaded_size > 0); + chassert(fs::file_size(getPathInLocalCache()) == downloaded_size); } void FileSegment::setDownloadFailedUnlocked(const FileSegmentGuard::Lock & lock) @@ -845,7 +852,8 @@ void FileSegment::detach(const FileSegmentGuard::Lock & lock, const LockedKey &) if (download_state == State::DETACHED) return; - resetDownloaderUnlocked(lock); + if (!downloader_id.empty()) + resetDownloaderUnlocked(lock); setDetachedState(lock); } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 843ffd45b63..9dff77e2af8 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -145,15 +145,12 @@ String CacheMetadata::getFileNameForFileSegment(size_t offset, FileSegmentKind s return std::to_string(offset) + file_suffix; } -String CacheMetadata::getPathInLocalCache(const Key & key, size_t offset, FileSegmentKind segment_kind) const +String CacheMetadata::getPathForFileSegment(const Key & key, size_t offset, FileSegmentKind segment_kind) const { - String file_suffix; - - const auto key_str = key.toString(); - return fs::path(path) / key_str.substr(0, 3) / key_str / getFileNameForFileSegment(offset, segment_kind); + return fs::path(getPathForKey(key)) / getFileNameForFileSegment(offset, segment_kind); } -String CacheMetadata::getPathInLocalCache(const Key & key) const +String CacheMetadata::getPathForKey(const Key & key) const { const auto key_str = key.toString(); return fs::path(path) / key_str.substr(0, 3) / key_str; @@ -178,7 +175,7 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( it = emplace( key, std::make_shared( - key, getPathInLocalCache(key), *cleanup_queue, is_initial_load)).first; + key, getPathForKey(key), *cleanup_queue, is_initial_load)).first; } key_metadata = it->second; @@ -260,7 +257,7 @@ void CacheMetadata::doCleanup() erase(it); LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); - const fs::path key_directory = getPathInLocalCache(cleanup_key); + const fs::path key_directory = getPathForKey(cleanup_key); const fs::path key_prefix_directory = key_directory.parent_path(); try @@ -370,8 +367,14 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm file_segment->queue_iterator->annul(); const auto path = key_metadata->getFileSegmentPath(*file_segment); - if (fs::exists(path)) + bool exists = fs::exists(path); + if (exists) + { + LOG_TEST(log, "Removed file segment at path: {}", path); fs::remove(path); + } + else if (file_segment->downloaded_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist"); file_segment->detach(segment_lock, *this); return key_metadata->erase(it); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 2e015b07ed0..a7e101c3d9d 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -85,12 +85,12 @@ public: const String & getBaseDirectory() const { return path; } - String getPathInLocalCache( + String getPathForFileSegment( const Key & key, size_t offset, FileSegmentKind segment_kind) const; - String getPathInLocalCache(const Key & key) const; + String getPathForKey(const Key & key) const; static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind); void iterate(IterateCacheMetadataFunc && func); From 5cb4363e58bf1553bd80a930779cda2a79ef34b0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Jun 2023 14:44:28 +0200 Subject: [PATCH 021/159] Remove assertion --- src/Interpreters/Cache/FileSegment.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index fb0ba0eba14..f3a21749086 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -313,12 +313,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) if (!size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing zero size is not allowed"); - auto file_segment_path = getPathInLocalCache(); - if (offset == range().left && fs::exists(file_segment_path)) - { - fs::remove(file_segment_path); - chassert(false); - } + const auto file_segment_path = getPathInLocalCache(); { auto lock = segment_guard.lock(); @@ -358,7 +353,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) "Cache writer was finalized (downloaded size: {}, state: {})", current_downloaded_size, stateToString(download_state)); - cache_writer = std::make_unique(getPathInLocalCache()); + cache_writer = std::make_unique(file_segment_path); } } @@ -385,6 +380,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) const auto file_size = fs::file_size(file_segment_path); chassert(downloaded_size <= file_size); chassert(reserved_size >= file_size); + chassert(file_size <= range().right + 1); if (downloaded_size != file_size) downloaded_size = file_size; } From 21e9877098c24fc90319fae2682a815da8fef389 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Jun 2023 15:08:04 +0200 Subject: [PATCH 022/159] Add assertion --- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index a60f5dffa96..561a66a826f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -932,18 +932,23 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); -#ifdef ABORT_ON_LOGICAL_ERROR const size_t new_file_offset = file_offset_of_buffer_end + size; - chassert(new_file_offset - 1 <= file_segment.range().right); const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); + if (new_file_offset > file_segment.range().right + 1) + { + auto file_segment_path = file_segment.getPathInLocalCache(); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Read unexpected size. File size: {}, file path: {}, file segment info: {}", + fs::file_size(file_segment_path), file_segment_path, file_segment.getInfoForLog()); + } if (new_file_offset > file_segment_write_offset) { - LOG_TRACE( - log, "Read {} bytes, file offset: {}, segment: {}, segment write offset: {}", + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Read unexpected size. Read {} bytes, file offset: {}, segment: {}, segment write offset: {}", size, file_offset_of_buffer_end, file_segment.range().toString(), file_segment_write_offset); - chassert(false); } -#endif } else { From d0da370d1e18c69e533f5d85725b7db4aa87d884 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 2 Jun 2023 16:19:24 +0200 Subject: [PATCH 023/159] Add a test for compound column identifier --- .../02771_resolve_compound_identifier.reference | 10 ++++++++++ .../0_stateless/02771_resolve_compound_identifier.sql | 11 +++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02771_resolve_compound_identifier.reference create mode 100644 tests/queries/0_stateless/02771_resolve_compound_identifier.sql diff --git a/tests/queries/0_stateless/02771_resolve_compound_identifier.reference b/tests/queries/0_stateless/02771_resolve_compound_identifier.reference new file mode 100644 index 00000000000..8b1acc12b63 --- /dev/null +++ b/tests/queries/0_stateless/02771_resolve_compound_identifier.reference @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/tests/queries/0_stateless/02771_resolve_compound_identifier.sql b/tests/queries/0_stateless/02771_resolve_compound_identifier.sql new file mode 100644 index 00000000000..db4d443379e --- /dev/null +++ b/tests/queries/0_stateless/02771_resolve_compound_identifier.sql @@ -0,0 +1,11 @@ +DROP DATABASE IF EXISTS test_02771; + +CREATE DATABASE test_02771; + +CREATE TABLE test_02771.t (x UInt8) ENGINE = MergeTree() ORDER BY x; + +INSERT INTO test_02771.t SELECT number FROM numbers(10); + +SELECT t.x FROM test_02771.t ORDER BY t.x; + +DROP DATABASE IF EXISTS test_02771; From 530f743ed062157cb0fc74d3bbc0bf51b186c0b5 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 2 Jun 2023 23:41:25 +0200 Subject: [PATCH 024/159] Fix Object data type for StorageDistributed --- src/Storages/StorageDistributed.cpp | 3 ++- src/Storages/StorageDummy.cpp | 3 ++- src/Storages/StorageDummy.h | 9 ++++++++- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 43b1333413e..b9625ce2ab7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -948,7 +948,8 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, auto resolved_remote_storage_id = remote_storage_id; if (remote_storage_id.hasDatabase()) resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); - auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns()); + + auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns(), distributed_storage_snapshot->object_columns); auto table_node = std::make_shared(std::move(storage), query_context); if (table_expression_modifiers) diff --git a/src/Storages/StorageDummy.cpp b/src/Storages/StorageDummy.cpp index e5f3b0b4d8e..4f2fb3883bf 100644 --- a/src/Storages/StorageDummy.cpp +++ b/src/Storages/StorageDummy.cpp @@ -9,8 +9,9 @@ namespace DB { -StorageDummy::StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_) +StorageDummy::StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, ColumnsDescription object_columns_) : IStorage(table_id_) + , object_columns(std::move(object_columns_)) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); diff --git a/src/Storages/StorageDummy.h b/src/Storages/StorageDummy.h index a7beef9d531..2f9a8beb4d0 100644 --- a/src/Storages/StorageDummy.h +++ b/src/Storages/StorageDummy.h @@ -11,7 +11,7 @@ namespace DB class StorageDummy : public IStorage { public: - StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_); + StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, ColumnsDescription object_columns_ = {}); std::string getName() const override { return "StorageDummy"; } @@ -22,6 +22,11 @@ public: bool supportsDynamicSubcolumns() const override { return true; } bool canMoveConditionsToPrewhere() const override { return false; } + StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const override + { + return std::make_shared(*this, metadata_snapshot, object_columns); + } + QueryProcessingStage::Enum getQueryProcessingStage( ContextPtr local_context, QueryProcessingStage::Enum to_stage, @@ -37,6 +42,8 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; +private: + const ColumnsDescription object_columns; }; class ReadFromDummy : public SourceStepWithFilter From 24e015b961189e1b2202cfdf5fefe6c9e5904e71 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 4 Jun 2023 13:34:20 +0200 Subject: [PATCH 025/159] Fix --- src/Interpreters/Cache/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 8a24a4fe5ee..7afd17d32d1 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -650,7 +650,7 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size) } ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedFileSegments); - ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->range().size()); + ProfileEvents::increment(ProfileEvents::FilesystemCacheEvictedBytes, segment->getDownloadedSize(false)); locked_key.removeFileSegment(segment->offset(), segment->lock()); return PriorityIterationResult::REMOVE_AND_CONTINUE; From 3657ef05fffa722115becb7f7e8937a3a472625d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:26:41 +0000 Subject: [PATCH 026/159] Cosmetics: Fix indentation --- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 30 +++++++++++--------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 3b1a41eb85d..b349c0567ef 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -14,19 +14,23 @@ namespace DB // mainly for serialization and deserialization of the index namespace ApproximateNearestNeighbour { - using AnnoyIndexThreadedBuildPolicy = ::Annoy::AnnoyIndexMultiThreadedBuildPolicy; - // TODO: Support different metrics. List of available metrics can be taken from here: - // https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 - template - class AnnoyIndex : public ::Annoy::AnnoyIndex - { - using Base = ::Annoy::AnnoyIndex; - public: - explicit AnnoyIndex(const uint64_t dim) : Base::AnnoyIndex(dim) {} - void serialize(WriteBuffer& ostr) const; - void deserialize(ReadBuffer& istr); - uint64_t getNumOfDimensions() const; - }; + +using AnnoyIndexThreadedBuildPolicy = ::Annoy::AnnoyIndexMultiThreadedBuildPolicy; + +// TODO: Support different metrics. List of available metrics can be taken from here: +// https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 +template +class AnnoyIndex : public ::Annoy::AnnoyIndex +{ + using Base = ::Annoy::AnnoyIndex; + +public: + explicit AnnoyIndex(const uint64_t dim) : Base::AnnoyIndex(dim) {} + void serialize(WriteBuffer& ostr) const; + void deserialize(ReadBuffer& istr); + uint64_t getNumOfDimensions() const; +}; + } template From 32756292309182bfa2ddf59213c1628a22651e26 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:31:34 +0000 Subject: [PATCH 027/159] Cosmetics: Remove parentheses in single statement if/for/while --- src/Storages/MergeTree/CommonANNIndexes.cpp | 94 ------------------- .../MergeTree/MergeTreeIndexAnnoy.cpp | 31 +----- 2 files changed, 4 insertions(+), 121 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 4b360e029e5..f0c6f256f73 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -35,17 +35,11 @@ void extractTargetVectorFromLiteral(ANN::ANNQueryInformation::Embedding & target for (const auto & value : literal.value()) { if (value.tryGet(float_element_of_target_vector)) - { target.emplace_back(float_element_of_target_vector); - } else if (value.tryGet(int_element_of_target_vector)) - { target.emplace_back(static_cast(int_element_of_target_vector)); - } else - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type of elements in target vector. Only float or int are supported."); - } } } @@ -74,9 +68,7 @@ ANNCondition::ANNCondition(const SelectQueryInfo & query_info, bool ANNCondition::alwaysUnknownOrTrue(String metric_name) const { if (!index_is_useful) - { return true; // Query isn't supported - } // If query is supported, check metrics for match return !(castMetricFromStringToType(metric_name) == query_information->metric); } @@ -85,72 +77,56 @@ float ANNCondition::getComparisonDistanceForWhereQuery() const { if (index_is_useful && query_information.has_value() && query_information->query_type == ANNQueryInformation::Type::Where) - { return query_information->distance; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported method for this query type"); } UInt64 ANNCondition::getLimit() const { if (index_is_useful && query_information.has_value()) - { return query_information->limit; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "No LIMIT section in query, not supported"); } std::vector ANNCondition::getTargetVector() const { if (index_is_useful && query_information.has_value()) - { return query_information->target; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Target vector was requested for useless or uninitialized index."); } size_t ANNCondition::getNumOfDimensions() const { if (index_is_useful && query_information.has_value()) - { return query_information->target.size(); - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of dimensions was requested for useless or uninitialized index."); } String ANNCondition::getColumnName() const { if (index_is_useful && query_information.has_value()) - { return query_information->column_name; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Column name was requested for useless or uninitialized index."); } ANNQueryInformation::Metric ANNCondition::getMetricType() const { if (index_is_useful && query_information.has_value()) - { return query_information->metric; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Metric name was requested for useless or uninitialized index."); } float ANNCondition::getPValueForLpDistance() const { if (index_is_useful && query_information.has_value()) - { return query_information->p_for_lp_dist; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "P from LPDistance was requested for useless or uninitialized index."); } ANNQueryInformation::Type ANNCondition::getQueryType() const { if (index_is_useful && query_information.has_value()) - { return query_information->query_type; - } throw Exception(ErrorCodes::LOGICAL_ERROR, "Query type was requested for useless or uninitialized index."); } @@ -171,24 +147,16 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) const auto & select = query.query->as(); if (select.prewhere()) // If query has PREWHERE clause - { traverseAST(select.prewhere(), rpn_prewhere_clause); - } if (select.where()) // If query has WHERE clause - { traverseAST(select.where(), rpn_where_clause); - } if (select.limitLength()) // If query has LIMIT clause - { traverseAtomAST(select.limitLength(), rpn_limit); - } if (select.orderBy()) // If query has ORDERBY clause - { traverseOrderByAST(select.orderBy(), rpn_order_by_clause); - } // Reverse RPNs for conveniences during parsing std::reverse(rpn_prewhere_clause.begin(), rpn_prewhere_clause.end()); @@ -203,29 +171,21 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) // Query without a LIMIT clause or with a limit greater than a restriction is not supported if (!limit_is_valid || limit_restriction < limit) - { return false; - } // Search type query in both sections isn't supported if (prewhere_is_valid && where_is_valid) - { return false; - } // Search type should be in WHERE or PREWHERE clause if (prewhere_is_valid || where_is_valid) - { query_information = std::move(prewhere_is_valid ? prewhere_info : where_info); - } if (order_by_is_valid) { // Query with valid where and order by type is not supported if (query_information.has_value()) - { return false; - } query_information = std::move(order_by_info); } @@ -244,17 +204,13 @@ void ANNCondition::traverseAST(const ASTPtr & node, RPN & rpn) const ASTs & children = func->arguments->children; // Traverse children nodes for (const auto& child : children) - { traverseAST(child, rpn); - } } RPNElement element; // Get the data behind node if (!traverseAtomAST(node, element)) - { element.function = RPNElement::FUNCTION_UNKNOWN; - } rpn.emplace_back(std::move(element)); } @@ -273,32 +229,20 @@ bool ANNCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) function->name == "cosineDistance" || function->name == "dotProduct" || function->name == "LpDistance") - { out.function = RPNElement::FUNCTION_DISTANCE; - } else if (function->name == "tuple") - { out.function = RPNElement::FUNCTION_TUPLE; - } else if (function->name == "array") - { out.function = RPNElement::FUNCTION_ARRAY; - } else if (function->name == "less" || function->name == "greater" || function->name == "lessOrEquals" || function->name == "greaterOrEquals") - { out.function = RPNElement::FUNCTION_COMPARISON; - } else if (function->name == "_CAST") - { out.function = RPNElement::FUNCTION_CAST; - } else - { return false; - } return true; } @@ -378,12 +322,8 @@ bool ANNCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) void ANNCondition::traverseOrderByAST(const ASTPtr & node, RPN & rpn) { if (const auto * expr_list = node->as()) - { if (const auto * order_by_element = expr_list->children.front()->as()) - { traverseAST(order_by_element->children.front(), rpn); - } - } } // Returns true and stores ANNQueryInformation if the query has valid WHERE clause @@ -395,17 +335,13 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) // WHERE section must have at least 5 expressions // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) if (rpn.size() < 5) - { return false; - } auto iter = rpn.begin(); // Query starts from operator less if (iter->function != RPNElement::FUNCTION_COMPARISON) - { return false; - } const bool greater_case = iter->func_name == "greater" || iter->func_name == "greaterOrEquals"; const bool less_case = iter->func_name == "less" || iter->func_name == "lessOrEquals"; @@ -415,9 +351,7 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) if (less_case) { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL) - { return false; - } expr.distance = getFloatOrIntLiteralOrPanic(iter); if (expr.distance < 0) @@ -427,22 +361,16 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) } else if (!greater_case) - { return false; - } auto end = rpn.end(); if (!matchMainParts(iter, end, expr)) - { return false; - } if (greater_case) { if (expr.target.size() < 2) - { return false; - } expr.distance = expr.target.back(); if (expr.distance < 0) throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", expr.distance); @@ -461,9 +389,7 @@ bool ANNCondition::matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr) // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) - { return false; - } auto iter = rpn.begin(); auto end = rpn.end(); @@ -490,9 +416,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en // Matches DistanceFunc->[Column]->[Tuple(array)Func]->TargetVector(floats)->[Column] if (iter->function != RPNElement::FUNCTION_DISTANCE) - { return false; - } expr.metric = castMetricFromStringToType(iter->func_name); ++iter; @@ -501,9 +425,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL && iter->function != RPNElement::FUNCTION_INT_LITERAL) - { return false; - } expr.p_for_lp_dist = getFloatOrIntLiteralOrPanic(iter); ++iter; } @@ -516,9 +438,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en } if (iter->function == RPNElement::FUNCTION_TUPLE || iter->function == RPNElement::FUNCTION_ARRAY) - { ++iter; - } if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { @@ -539,9 +459,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en ++iter; /// Cast should be made to array or tuple if (!iter->func_name.starts_with("Array") && !iter->func_name.starts_with("Tuple")) - { return false; - } ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { @@ -554,31 +472,23 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en ++iter; } else - { return false; - } } while (iter != end) { if (iter->function == RPNElement::FUNCTION_FLOAT_LITERAL || iter->function == RPNElement::FUNCTION_INT_LITERAL) - { expr.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); - } else if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { if (identifier_found) - { return false; - } expr.column_name = std::move(iter->identifier.value()); identifier_found = true; } else - { return false; - } ++iter; } @@ -591,13 +501,9 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en float ANNCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) { if (iter->float_literal.has_value()) - { return iter->float_literal.value(); - } if (iter->int_literal.has_value()) - { return static_cast(iter->int_literal.value()); - } throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong parsed AST in buildRPN\n"); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index f64d6104ac6..0b7e1f29f03 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -219,17 +219,11 @@ bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { if (distance_name == "L2Distance") - { return getUsefulRangesImpl<::Annoy::Euclidean>(idx_granule); - } else if (distance_name == "cosineDistance") - { return getUsefulRangesImpl<::Annoy::Angular>(idx_granule); - } else - { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); - } } @@ -297,26 +291,18 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") - { return std::make_shared >(index.name, index.sample_block); - } - if (distance_name == "cosineDistance") - { + else if (distance_name == "cosineDistance") return std::make_shared >(index.name, index.sample_block); - } throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { if (distance_name == "L2Distance") - { return std::make_shared >(index.name, index.sample_block, number_of_trees); - } if (distance_name == "cosineDistance") - { return std::make_shared >(index.name, index.sample_block, number_of_trees); - } throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -331,16 +317,10 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) uint64_t param = 100; String distance_name = "L2Distance"; if (!index.arguments.empty() && !index.arguments[0].tryGet(param)) - { if (!index.arguments[0].tryGet(distance_name)) - { throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse first argument"); - } - } if (index.arguments.size() > 1 && !index.arguments[1].tryGet(distance_name)) - { throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse second argument"); - } return std::make_shared(index, param, distance_name); } @@ -381,18 +361,14 @@ static void assertIndexColumnsType(const Block & header) void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { if (index.arguments.size() > 2) - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); - } + if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 && index.arguments[0].getType() != Field::Types::String) - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String."); - } + if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - { throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String."); - } if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); @@ -401,4 +377,5 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) } } + #endif // ENABLE_ANNOY From edad92a7f224732c52f9d57d062e46c23396d19a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:36:27 +0000 Subject: [PATCH 028/159] Cosmetics: Minor aesthetic fixes --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 45 ++++++++++--------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 20 +++------ 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 0b7e1f29f03..b31779ff71c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -18,10 +18,10 @@ namespace DB namespace ApproximateNearestNeighbour { -template -void AnnoyIndex::serialize(WriteBuffer& ostr) const +template +void AnnoyIndex::serialize(WriteBuffer& ostr) const { - assert(Base::_built); + chassert(Base::_built); writeIntBinary(Base::_s, ostr); writeIntBinary(Base::_n_items, ostr); writeIntBinary(Base::_n_nodes, ostr); @@ -32,10 +32,10 @@ void AnnoyIndex::serialize(WriteBuffer& ostr) const ostr.write(reinterpret_cast(Base::_nodes), Base::_s * Base::_n_nodes); } -template -void AnnoyIndex::deserialize(ReadBuffer& istr) +template +void AnnoyIndex::deserialize(ReadBuffer& istr) { - assert(!Base::_built); + chassert(!Base::_built); readIntBinary(Base::_s, istr); readIntBinary(Base::_n_items, istr); readIntBinary(Base::_n_nodes, istr); @@ -54,8 +54,8 @@ void AnnoyIndex::deserialize(ReadBuffer& istr) Base::_built = true; } -template -uint64_t AnnoyIndex::getNumOfDimensions() const +template +uint64_t AnnoyIndex::getNumOfDimensions() const { return Base::get_f(); } @@ -84,16 +84,16 @@ template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy( const String & index_name_, const Block & index_sample_block_, - AnnoyIndexPtr index_base_) + AnnoyIndexPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) - , index(std::move(index_base_)) + , index(std::move(index_)) {} template void MergeTreeIndexGranuleAnnoy::serializeBinary(WriteBuffer & ostr) const { - /// number of dimensions is required in the constructor, + /// Number of dimensions is required in the index constructor, /// so it must be written and read separately from the other part writeIntBinary(index->getNumOfDimensions(), ostr); // write dimension index->serialize(ostr); @@ -123,7 +123,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndR { // NOLINTNEXTLINE(*) index->build(static_cast(number_of_trees), /*number_of_threads=*/1); - auto granule = std::make_shared >(index_name, index_sample_block, index); + auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; return granule; } @@ -202,7 +202,8 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const SelectQueryInfo & query, ContextPtr context, const String& distance_name_) - : condition(query, context), distance_name(distance_name_) + : condition(query, context) + , distance_name(distance_name_) {} @@ -232,15 +233,16 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI { UInt64 limit = condition.getLimit(); UInt64 index_granularity = condition.getIndexGranularity(); - std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighbour::ANNQueryInformation::Type::Where ? - std::optional(condition.getComparisonDistanceForWhereQuery()) : std::nullopt; + std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighbour::ANNQueryInformation::Type::Where + ? std::optional(condition.getComparisonDistanceForWhereQuery()) + : std::nullopt; if (comp_dist && comp_dist.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); std::vector target_vec = condition.getTargetVector(); - auto granule = std::dynamic_pointer_cast >(idx_granule); + auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); @@ -291,18 +293,19 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") - return std::make_shared >(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); else if (distance_name == "cosineDistance") - return std::make_shared >(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { + /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_name == "L2Distance") - return std::make_shared >(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, number_of_trees); if (distance_name == "cosineDistance") - return std::make_shared >(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, number_of_trees); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -378,4 +381,4 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) } -#endif // ENABLE_ANNOY +#endif diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index b349c0567ef..d591187fc64 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -15,17 +15,13 @@ namespace DB namespace ApproximateNearestNeighbour { -using AnnoyIndexThreadedBuildPolicy = ::Annoy::AnnoyIndexMultiThreadedBuildPolicy; - -// TODO: Support different metrics. List of available metrics can be taken from here: -// https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 template -class AnnoyIndex : public ::Annoy::AnnoyIndex +class AnnoyIndex : public ::Annoy::AnnoyIndex { - using Base = ::Annoy::AnnoyIndex; + using Base = ::Annoy::AnnoyIndex; public: - explicit AnnoyIndex(const uint64_t dim) : Base::AnnoyIndex(dim) {} + explicit AnnoyIndex(uint64_t dim) : Base::AnnoyIndex(dim) {} void serialize(WriteBuffer& ostr) const; void deserialize(ReadBuffer& istr); uint64_t getNumOfDimensions() const; @@ -40,10 +36,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule using AnnoyIndexPtr = std::shared_ptr; MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleAnnoy( - const String & index_name_, - const Block & index_sample_block_, - AnnoyIndexPtr index_base_); + MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); ~MergeTreeIndexGranuleAnnoy() override = default; @@ -118,8 +111,7 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator() const override; - MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } @@ -131,4 +123,4 @@ private: } -#endif // ENABLE_ANNOY +#endif From 1018677f464756f0fcb4a1c26e7143e07d30dfe7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:39:04 +0000 Subject: [PATCH 029/159] Cosmetics: Move ctors into cpp file --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++++++++ src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 8 ++------ 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b31779ff71c..e82f279c412 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -18,6 +18,12 @@ namespace DB namespace ApproximateNearestNeighbour { +template +AnnoyIndex::AnnoyIndex(uint64_t dim) + : Base::AnnoyIndex(dim) +{ +} + template void AnnoyIndex::serialize(WriteBuffer& ostr) const { @@ -290,6 +296,12 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI return result_vector; } +MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_) + : IMergeTreeIndex(index_) + , number_of_trees(number_of_trees_) + , distance_name(distance_name_) +{} + MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index d591187fc64..2ccb3527b18 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -21,7 +21,7 @@ class AnnoyIndex : public ::Annoy::AnnoyIndex; public: - explicit AnnoyIndex(uint64_t dim) : Base::AnnoyIndex(dim) {} + explicit AnnoyIndex(uint64_t dim); void serialize(WriteBuffer& ostr) const; void deserialize(ReadBuffer& istr); uint64_t getNumOfDimensions() const; @@ -100,11 +100,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex { public: - MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_) - : IMergeTreeIndex(index_) - , number_of_trees(number_of_trees_) - , distance_name(distance_name_) - {} + MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_); ~MergeTreeIndexAnnoy() override = default; From 4631595cf658ae007e70b563a4631b2a95fd2439 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:44:20 +0000 Subject: [PATCH 030/159] Cosmetics: number_of_trees --> trees --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 14 +++++++------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e82f279c412..946889066b1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -118,17 +118,17 @@ template MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( const String & index_name_, const Block & index_sample_block_, - uint64_t number_of_trees_) + uint64_t trees_) : index_name(index_name_) , index_sample_block(index_sample_block_) - , number_of_trees(number_of_trees_) + , trees(trees_) {} template MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndReset() { // NOLINTNEXTLINE(*) - index->build(static_cast(number_of_trees), /*number_of_threads=*/1); + index->build(static_cast(trees), /*number_of_threads=*/1); auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; return granule; @@ -296,9 +296,9 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI return result_vector; } -MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_) +MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_) : IMergeTreeIndex(index_) - , number_of_trees(number_of_trees_) + , trees(trees_) , distance_name(distance_name_) {} @@ -315,9 +315,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_name == "L2Distance") - return std::make_shared>(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, trees); if (distance_name == "cosineDistance") - return std::make_shared>(index.name, index.sample_block, number_of_trees); + return std::make_shared>(index.name, index.sample_block, trees); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 2ccb3527b18..2c41cf457ce 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -56,7 +56,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator using AnnoyIndex = ApproximateNearestNeighbour::AnnoyIndex; using AnnoyIndexPtr = std::shared_ptr; - MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t number_of_trees); + MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees); ~MergeTreeIndexAggregatorAnnoy() override = default; bool empty() const override { return !index || index->get_n_items() == 0; } @@ -65,7 +65,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator String index_name; Block index_sample_block; - const uint64_t number_of_trees; + const uint64_t trees; AnnoyIndexPtr index; }; @@ -100,7 +100,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex { public: - MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t number_of_trees_, const String& distance_name_); + MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_); ~MergeTreeIndexAnnoy() override = default; @@ -112,7 +112,7 @@ public: bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } private: - const uint64_t number_of_trees; + const uint64_t trees; const String distance_name; }; From a8bf7af2918bf3101600cec4465c20e3a3faec0b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:47:05 +0000 Subject: [PATCH 031/159] Cosmetics: Move ErrorCodes to top --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 946889066b1..54df6f46ef2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -15,6 +15,16 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int INCORRECT_DATA; + extern const int INCORRECT_NUMBER_OF_COLUMNS; + extern const int INCORRECT_QUERY; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; +} + namespace ApproximateNearestNeighbour { @@ -69,16 +79,6 @@ uint64_t AnnoyIndex::getNumOfDimensions() const } -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int INCORRECT_DATA; - extern const int INCORRECT_NUMBER_OF_COLUMNS; - extern const int INCORRECT_QUERY; - extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; -} - template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_) : index_name(index_name_) From f8c1f2bd33898e2fbc1bb4a963458a781778f722 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:48:12 +0000 Subject: [PATCH 032/159] Cosmetics: Remove absolute namespace qualification of Annoy library internals --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 54df6f46ef2..716624b7453 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -226,9 +226,9 @@ bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { if (distance_name == "L2Distance") - return getUsefulRangesImpl<::Annoy::Euclidean>(idx_granule); + return getUsefulRangesImpl(idx_granule); else if (distance_name == "cosineDistance") - return getUsefulRangesImpl<::Annoy::Angular>(idx_granule); + return getUsefulRangesImpl(idx_granule); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -305,9 +305,9 @@ MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64 MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { if (distance_name == "L2Distance") - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); else if (distance_name == "cosineDistance") - return std::make_shared>(index.name, index.sample_block); + return std::make_shared>(index.name, index.sample_block); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } @@ -315,9 +315,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_name == "L2Distance") - return std::make_shared>(index.name, index.sample_block, trees); + return std::make_shared>(index.name, index.sample_block, trees); if (distance_name == "cosineDistance") - return std::make_shared>(index.name, index.sample_block, trees); + return std::make_shared>(index.name, index.sample_block, trees); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); } From e373cf682ade101bcc2fd288263e547690834ff8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:50:59 +0000 Subject: [PATCH 033/159] Cosmetics: Unwrap Annoy index from nested namespace --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ---- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 11 ++--------- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 716624b7453..cbeb6540721 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -25,8 +25,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace ApproximateNearestNeighbour -{ template AnnoyIndex::AnnoyIndex(uint64_t dim) @@ -76,8 +74,6 @@ uint64_t AnnoyIndex::getNumOfDimensions() const return Base::get_f(); } -} - template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 2c41cf457ce..44f5f8d8eb7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -10,11 +10,6 @@ namespace DB { -// auxiliary namespace for working with spotify-annoy library -// mainly for serialization and deserialization of the index -namespace ApproximateNearestNeighbour -{ - template class AnnoyIndex : public ::Annoy::AnnoyIndex { @@ -27,12 +22,10 @@ public: uint64_t getNumOfDimensions() const; }; -} - template struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule { - using AnnoyIndex = ApproximateNearestNeighbour::AnnoyIndex; + using AnnoyIndex = AnnoyIndex; using AnnoyIndexPtr = std::shared_ptr; MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); @@ -53,7 +46,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule template struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator { - using AnnoyIndex = ApproximateNearestNeighbour::AnnoyIndex; + using AnnoyIndex = AnnoyIndex; using AnnoyIndexPtr = std::shared_ptr; MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees); From dc88d4e7422e9b0c19873297a9b7e2d3107ed052 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:53:07 +0000 Subject: [PATCH 034/159] Cosmetics: Factorize repeated typedefs into a single typedef --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 15 ++++++--------- 2 files changed, 10 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index cbeb6540721..58c59028075 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -86,7 +86,7 @@ template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy( const String & index_name_, const Block & index_sample_block_, - AnnoyIndexPtr index_) + AnnoyIndexPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) , index(std::move(index_)) @@ -106,7 +106,7 @@ void MergeTreeIndexGranuleAnnoy::deserializeBinary(ReadBuffer & istr, { uint64_t dimension; readIntBinary(dimension, istr); - index = std::make_shared(dimension); + index = std::make_shared>(dimension); index->deserialize(istr); } @@ -164,7 +164,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (offsets[i + 1] - offsets[i] != size) throw Exception(ErrorCodes::INCORRECT_DATA, "Arrays should have same length"); - index = std::make_shared(size); + index = std::make_shared>(size); index->add_item(index->get_n_items(), array.data()); /// add all rows from 1 to num_rows - 1 (this is the same as the beginning of the last element) @@ -190,7 +190,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t } assert(!data.empty()); if (!index) - index = std::make_shared(data[0].size()); + index = std::make_shared>(data[0].size()); for (const auto& item : data) index->add_item(index->get_n_items(), item.data()); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 44f5f8d8eb7..bb0d1883fc2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -22,14 +22,14 @@ public: uint64_t getNumOfDimensions() const; }; +template +using AnnoyIndexPtr = std::shared_ptr>; + template struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule { - using AnnoyIndex = AnnoyIndex; - using AnnoyIndexPtr = std::shared_ptr; - MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); + MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); ~MergeTreeIndexGranuleAnnoy() override = default; @@ -40,15 +40,12 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule String index_name; Block index_sample_block; - AnnoyIndexPtr index; + AnnoyIndexPtr index; }; template struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator { - using AnnoyIndex = AnnoyIndex; - using AnnoyIndexPtr = std::shared_ptr; - MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, uint64_t trees); ~MergeTreeIndexAggregatorAnnoy() override = default; @@ -59,7 +56,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator String index_name; Block index_sample_block; const uint64_t trees; - AnnoyIndexPtr index; + AnnoyIndexPtr index; }; From 594572b0de48f7bd3be39f3abcb057f708b7fcf9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 19:55:01 +0000 Subject: [PATCH 035/159] Cosmetics: AnnoyIndex --> AnnoyIndexWithSerialization --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 16 ++++++++-------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 12 ++++++------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 58c59028075..60b9efcaf67 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -27,13 +27,13 @@ namespace ErrorCodes template -AnnoyIndex::AnnoyIndex(uint64_t dim) +AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(uint64_t dim) : Base::AnnoyIndex(dim) { } template -void AnnoyIndex::serialize(WriteBuffer& ostr) const +void AnnoyIndexWithSerialization::serialize(WriteBuffer& ostr) const { chassert(Base::_built); writeIntBinary(Base::_s, ostr); @@ -47,7 +47,7 @@ void AnnoyIndex::serialize(WriteBuffer& ostr) const } template -void AnnoyIndex::deserialize(ReadBuffer& istr) +void AnnoyIndexWithSerialization::deserialize(ReadBuffer& istr) { chassert(!Base::_built); readIntBinary(Base::_s, istr); @@ -69,7 +69,7 @@ void AnnoyIndex::deserialize(ReadBuffer& istr) } template -uint64_t AnnoyIndex::getNumOfDimensions() const +uint64_t AnnoyIndexWithSerialization::getNumOfDimensions() const { return Base::get_f(); } @@ -86,7 +86,7 @@ template MergeTreeIndexGranuleAnnoy::MergeTreeIndexGranuleAnnoy( const String & index_name_, const Block & index_sample_block_, - AnnoyIndexPtr index_) + AnnoyIndexWithSerializationPtr index_) : index_name(index_name_) , index_sample_block(index_sample_block_) , index(std::move(index_)) @@ -106,7 +106,7 @@ void MergeTreeIndexGranuleAnnoy::deserializeBinary(ReadBuffer & istr, { uint64_t dimension; readIntBinary(dimension, istr); - index = std::make_shared>(dimension); + index = std::make_shared>(dimension); index->deserialize(istr); } @@ -164,7 +164,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (offsets[i + 1] - offsets[i] != size) throw Exception(ErrorCodes::INCORRECT_DATA, "Arrays should have same length"); - index = std::make_shared>(size); + index = std::make_shared>(size); index->add_item(index->get_n_items(), array.data()); /// add all rows from 1 to num_rows - 1 (this is the same as the beginning of the last element) @@ -190,7 +190,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t } assert(!data.empty()); if (!index) - index = std::make_shared>(data[0].size()); + index = std::make_shared>(data[0].size()); for (const auto& item : data) index->add_item(index->get_n_items(), item.data()); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index bb0d1883fc2..c5520ab5673 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -11,25 +11,25 @@ namespace DB { template -class AnnoyIndex : public ::Annoy::AnnoyIndex +class AnnoyIndexWithSerialization : public ::Annoy::AnnoyIndex { using Base = ::Annoy::AnnoyIndex; public: - explicit AnnoyIndex(uint64_t dim); + explicit AnnoyIndexWithSerialization(uint64_t dim); void serialize(WriteBuffer& ostr) const; void deserialize(ReadBuffer& istr); uint64_t getNumOfDimensions() const; }; template -using AnnoyIndexPtr = std::shared_ptr>; +using AnnoyIndexWithSerializationPtr = std::shared_ptr>; template struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule { MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexPtr index_); + MergeTreeIndexGranuleAnnoy(const String & index_name_, const Block & index_sample_block_, AnnoyIndexWithSerializationPtr index_); ~MergeTreeIndexGranuleAnnoy() override = default; @@ -40,7 +40,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule String index_name; Block index_sample_block; - AnnoyIndexPtr index; + AnnoyIndexWithSerializationPtr index; }; template @@ -56,7 +56,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator String index_name; Block index_sample_block; const uint64_t trees; - AnnoyIndexPtr index; + AnnoyIndexWithSerializationPtr index; }; From c5ededdc5badab8fd18c6a321008157f2d1bbad5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:03:51 +0000 Subject: [PATCH 036/159] Cosmetics: Switch arguments in MTIConditionAnnoy ctor --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 60b9efcaf67..b7346e540d2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -202,8 +202,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const IndexDescription & /*index*/, const SelectQueryInfo & query, - ContextPtr context, - const String& distance_name_) + const String& distance_name_, + ContextPtr context) : condition(query, context) , distance_name(distance_name_) {} @@ -320,7 +320,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition( const SelectQueryInfo & query, ContextPtr context) const { - return std::make_shared(index, query, context, distance_name); + return std::make_shared(index, query, distance_name, context); }; MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index c5520ab5673..9ebaa335542 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -66,8 +66,8 @@ public: MergeTreeIndexConditionAnnoy( const IndexDescription & index, const SelectQueryInfo & query, - ContextPtr context, - const String& distance_name); + const String& distance_name, + ContextPtr context); bool alwaysUnknownOrTrue() const override; From 3b77e4090221b9977fc98bef1e7fac5cf035eb6a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:05:25 +0000 Subject: [PATCH 037/159] Cosmetics: Remove dots from exception messages --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b7346e540d2..e13ea91e56c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -345,7 +345,7 @@ static void assertIndexColumnsType(const Block & header) if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported.", + "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); } else if (const auto * tuple_type = typeid_cast(column_data_type_ptr.get())) @@ -357,14 +357,14 @@ static void assertIndexColumnsType(const Block & header) if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported.", + "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); } } else throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported.", + "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); } @@ -376,10 +376,10 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String."); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String."); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); From 423f69228239503b420153054a7c878b14aa2f47 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:07:43 +0000 Subject: [PATCH 038/159] Cosmetics: Remove unnecessary toString() --- src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexInverted.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexMinMax.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp index fe5a2a861f6..ef98accfbc6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp @@ -42,7 +42,7 @@ void MergeTreeIndexAggregatorBloomFilter::update(const Block & block, size_t * p { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); Block granule_index_block; size_t max_read_rows = std::min(block.rows() - *pos, limit); diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e13ea91e56c..133c0a9a58a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -137,7 +137,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t throw Exception( ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. Position: {}, Block rows: {}.", - toString(*pos), toString(block.rows())); + *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); if (rows_read == 0) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 06fddd51cb8..b15bf4d6811 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -92,7 +92,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index baa11368c8b..e19187646cd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -123,7 +123,7 @@ void MergeTreeIndexAggregatorInverted::update(const Block & block, size_t * pos, { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); auto row_id = store->getNextRowIDRange(rows_read); diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index d80f7521430..3b011837cb3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -122,7 +122,7 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 901636a2de9..120b3e43472 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -146,7 +146,7 @@ void MergeTreeIndexAggregatorSet::update(const Block & block, size_t * pos, size { if (*pos >= block.rows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "The provided position is not less than the number of block rows. " - "Position: {}, Block rows: {}.", toString(*pos), toString(block.rows())); + "Position: {}, Block rows: {}.", *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); From 7608e08eed5cb80f6be097ca572827a5090a0469 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:09:03 +0000 Subject: [PATCH 039/159] Cosmetics: more constness --- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 9ebaa335542..cde61af2891 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -38,8 +38,8 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule bool empty() const override { return !index.get(); } - String index_name; - Block index_sample_block; + const String index_name; + const Block index_sample_block; AnnoyIndexWithSerializationPtr index; }; @@ -53,8 +53,8 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator MergeTreeIndexGranulePtr getGranuleAndReset() override; void update(const Block & block, size_t * pos, size_t limit) override; - String index_name; - Block index_sample_block; + const String index_name; + const Block index_sample_block; const uint64_t trees; AnnoyIndexWithSerializationPtr index; }; @@ -81,7 +81,7 @@ private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; - ApproximateNearestNeighbour::ANNCondition condition; + const ApproximateNearestNeighbour::ANNCondition condition; const String distance_name; }; From a3d4ede26cc57b80a74061072a8b4e7fbe2832bf Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:11:10 +0000 Subject: [PATCH 040/159] Cosmetics: Update exception messages --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 133c0a9a58a..cd094bbeeac 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -144,7 +144,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t return; if (index_sample_block.columns() > 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one column is supported"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); auto index_column_name = index_sample_block.getByPosition(0).name; const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); @@ -162,7 +162,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t size_t size = offsets[0]; for (size_t i = 0; i < num_rows - 1; ++i) if (offsets[i + 1] - offsets[i] != size) - throw Exception(ErrorCodes::INCORRECT_DATA, "Arrays should have same length"); + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); index = std::make_shared>(size); From 2b74daaa1744588c44414f5e87578107fd0e1b84 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:12:29 +0000 Subject: [PATCH 041/159] Cosmetics: make input switch a bit more idiomatic --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index cd094bbeeac..048200eb57f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -148,8 +148,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t auto index_column_name = index_sample_block.getByPosition(0).name; const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); - const auto & column_array = typeid_cast(column_cut.get()); - if (column_array) + + if (const auto & column_array = typeid_cast(column_cut.get())) { const auto & data = column_array->getData(); const auto & array = typeid_cast(data).getData(); @@ -171,14 +171,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (size_t current_row = 1; current_row < num_rows; ++current_row) index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); } - else + else if (const auto & column_tuple = typeid_cast(column_cut.get())) { - /// Other possible type of column is Tuple - const auto & column_tuple = typeid_cast(column_cut.get()); - - if (!column_tuple) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type was given to index."); - const auto & columns = column_tuple->getColumns(); std::vector> data{column_tuple->size(), std::vector()}; @@ -194,6 +188,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (const auto& item : data) index->add_item(index->get_n_items(), item.data()); } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column"); *pos += rows_read; } From 68ad903f4c9a6bc04f6847d586dd6e8ff08c4aae Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:13:22 +0000 Subject: [PATCH 042/159] Cosmetics: unglue * and & --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 048200eb57f..80c56a299be 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -149,10 +149,10 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t auto index_column_name = index_sample_block.getByPosition(0).name; const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); - if (const auto & column_array = typeid_cast(column_cut.get())) + if (const auto & column_array = typeid_cast(column_cut.get())) { const auto & data = column_array->getData(); - const auto & array = typeid_cast(data).getData(); + const auto & array = typeid_cast(data).getData(); if (array.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); const auto & offsets = column_array->getOffsets(); @@ -171,21 +171,21 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (size_t current_row = 1; current_row < num_rows; ++current_row) index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); } - else if (const auto & column_tuple = typeid_cast(column_cut.get())) + else if (const auto & column_tuple = typeid_cast(column_cut.get())) { const auto & columns = column_tuple->getColumns(); std::vector> data{column_tuple->size(), std::vector()}; - for (const auto& column : columns) + for (const auto & column : columns) { - const auto& pod_array = typeid_cast(column.get())->getData(); + const auto & pod_array = typeid_cast(column.get())->getData(); for (size_t i = 0; i < pod_array.size(); ++i) data[i].push_back(pod_array[i]); } assert(!data.empty()); if (!index) index = std::make_shared>(data[0].size()); - for (const auto& item : data) + for (const auto & item : data) index->add_item(index->get_n_items(), item.data()); } else From 15c9e235c44785a8dc67544728d607271fe10436 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:19:44 +0000 Subject: [PATCH 043/159] Cosmetics: add some comments + minor changes --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 80c56a299be..ea284892754 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -140,6 +140,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t *pos, block.rows()); size_t rows_read = std::min(limit, block.rows() - *pos); + if (rows_read == 0) return; @@ -153,10 +154,12 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t { const auto & data = column_array->getData(); const auto & array = typeid_cast(data).getData(); + if (array.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); + const auto & offsets = column_array->getOffsets(); - size_t num_rows = offsets.size(); + const size_t num_rows = offsets.size(); /// Check all sizes are the same size_t size = offsets[0]; @@ -166,8 +169,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t index = std::make_shared>(size); + /// Add all rows of block index->add_item(index->get_n_items(), array.data()); - /// add all rows from 1 to num_rows - 1 (this is the same as the beginning of the last element) for (size_t current_row = 1; current_row < num_rows; ++current_row) index->add_item(index->get_n_items(), &array[offsets[current_row - 1]]); } @@ -175,6 +178,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t { const auto & columns = column_tuple->getColumns(); + /// TODO check if calling index->add_item() directly on the block's tuples is faster than materializing everything std::vector> data{column_tuple->size(), std::vector()}; for (const auto & column : columns) { @@ -182,9 +186,12 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t for (size_t i = 0; i < pod_array.size(); ++i) data[i].push_back(pod_array[i]); } - assert(!data.empty()); - if (!index) - index = std::make_shared>(data[0].size()); + + if (data.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read); + + index = std::make_shared>(data[0].size()); + for (const auto & item : data) index->add_item(index->get_n_items(), item.data()); } From 828155ebefde62b610e9866a45a6d7fd71eb14b7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:21:47 +0000 Subject: [PATCH 044/159] Cosmetics: Move assertIndexColumnsType() into annoyIndexValidator() --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index ea284892754..636fd384248 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -338,9 +338,22 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) return std::make_shared(index, param, distance_name); } -static void assertIndexColumnsType(const Block & header) +void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { - DataTypePtr column_data_type_ptr = header.getDataTypes()[0]; + if (index.arguments.size() > 2) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); + + if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 + && index.arguments[0].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); + + if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); + + if (index.column_names.size() != 1 || index.data_types.size() != 1) + throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + + DataTypePtr column_data_type_ptr = index.sample_block.getDataTypes()[0]; if (const auto * array_type = typeid_cast(column_data_type_ptr.get())) { @@ -369,25 +382,6 @@ static void assertIndexColumnsType(const Block & header) ErrorCodes::ILLEGAL_COLUMN, "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", column_data_type_ptr->getName()); - -} - -void annoyIndexValidator(const IndexDescription & index, bool /* attach */) -{ - if (index.arguments.size() > 2) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); - - if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 - && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); - - if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); - - if (index.column_names.size() != 1 || index.data_types.size() != 1) - throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); - - assertIndexColumnsType(index.sample_block); } } From 0854d913723d967e07105857924f788817ac48c5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:27:01 +0000 Subject: [PATCH 045/159] Cosmetics: Rename variable --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 636fd384248..d75f4978f08 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -353,35 +353,35 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); - DataTypePtr column_data_type_ptr = index.sample_block.getDataTypes()[0]; + DataTypePtr data_type = index.sample_block.getDataTypes()[0]; - if (const auto * array_type = typeid_cast(column_data_type_ptr.get())) + if (const auto * data_type_array = typeid_cast(data_type.get())) { - TypeIndex nested_type_index = array_type->getNestedType()->getTypeId(); + TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - column_data_type_ptr->getName()); + data_type->getName()); } - else if (const auto * tuple_type = typeid_cast(column_data_type_ptr.get())) + else if (const auto * data_type_tuple = typeid_cast(data_type.get())) { - const DataTypes & nested_types = tuple_type->getElements(); - for (const auto & type : nested_types) + const DataTypes & inner_types = data_type_tuple->getElements(); + for (const auto & inner_type : inner_types) { - TypeIndex nested_type_index = type->getTypeId(); + TypeIndex nested_type_index = inner_type->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - column_data_type_ptr->getName()); + "Unexpected inner_type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", + data_type->getName()); } } else throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - column_data_type_ptr->getName()); + data_type->getName()); } } From 62c8b9a7a11c0a3aae75a10a2a0ad61db5d63b55 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:28:20 +0000 Subject: [PATCH 046/159] Cosmetics: Factorize throw into lambda --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d75f4978f08..d35f435c391 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -353,16 +353,21 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type) + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Annoy indexes can only be created on columns of type Array(Float32) and Tuple(Float32). Given type: {}", + data_type->getName()); + }; + DataTypePtr data_type = index.sample_block.getDataTypes()[0]; if (const auto * data_type_array = typeid_cast(data_type.get())) { TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - data_type->getName()); + throw_unsupported_underlying_column_exception(data_type); } else if (const auto * data_type_tuple = typeid_cast(data_type.get())) { @@ -371,17 +376,11 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { TypeIndex nested_type_index = inner_type->getTypeId(); if (!WhichDataType(nested_type_index).isFloat32()) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Unexpected inner_type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - data_type->getName()); + throw_unsupported_underlying_column_exception(data_type); } } else - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Unexpected type {} of Annoy index. Only Array(Float32) and Tuple(Float32) are supported", - data_type->getName()); + throw_unsupported_underlying_column_exception(data_type); } } From 5d871c7fa09637eb89b69680ba5e5d256bddbdd7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:29:51 +0000 Subject: [PATCH 047/159] Cosmetics: +comments --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d35f435c391..8f0cad48dc0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -340,6 +340,8 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) void annoyIndexValidator(const IndexDescription & index, bool /* attach */) { + /// Check number and type of Annoy index arguments: + if (index.arguments.size() > 2) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); @@ -350,9 +352,13 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); + /// Check that the index is created on a single column + if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + /// Check data type of indexed column: + auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type) { throw Exception( From f577bf35fc53262290e3c6e18352d1a446cbb642 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:35:38 +0000 Subject: [PATCH 048/159] Simplify Annoy parameterization --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 25 ++++--- tests/queries/0_stateless/02354_annoy.sh | 73 +------------------ .../0_stateless/02354_annoy_index.reference | 1 + .../queries/0_stateless/02354_annoy_index.sql | 26 +++++++ 4 files changed, 42 insertions(+), 83 deletions(-) create mode 100644 tests/queries/0_stateless/02354_annoy_index.reference create mode 100644 tests/queries/0_stateless/02354_annoy_index.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 8f0cad48dc0..dc353c97143 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -328,14 +328,16 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition( MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { - uint64_t param = 100; + uint64_t trees = 100; String distance_name = "L2Distance"; - if (!index.arguments.empty() && !index.arguments[0].tryGet(param)) - if (!index.arguments[0].tryGet(distance_name)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse first argument"); - if (index.arguments.size() > 1 && !index.arguments[1].tryGet(distance_name)) - throw Exception(ErrorCodes::INCORRECT_DATA, "Can't parse second argument"); - return std::make_shared(index, param, distance_name); + + if (!index.arguments.empty()) + distance_name = index.arguments[0].get(); + + if (index.arguments.size() > 1) + trees = index.arguments[1].get(); + + return std::make_shared(index, trees, distance_name); } void annoyIndexValidator(const IndexDescription & index, bool /* attach */) @@ -345,12 +347,11 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.arguments.size() > 2) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index must not have more than two parameters"); - if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::UInt64 - && index.arguments[0].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index first argument must be UInt64 or String"); + if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance function argument of Annoy index must be of type String"); - if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::String) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index second argument must be String"); + if (index.arguments.size() > 1 && index.arguments[1].getType() != Field::Types::UInt64) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Number of trees argument of Annoy index must be UInt64"); /// Check that the index is created on a single column diff --git a/tests/queries/0_stateless/02354_annoy.sh b/tests/queries/0_stateless/02354_annoy.sh index 87258debf0f..1031ea81946 100755 --- a/tests/queries/0_stateless/02354_annoy.sh +++ b/tests/queries/0_stateless/02354_annoy.sh @@ -91,7 +91,7 @@ CREATE TABLE 02354_annoy_cosine ( id Int32, embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy(100, 'cosineDistance') GRANULARITY 1 + INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 ) ENGINE = MergeTree ORDER BY id @@ -120,7 +120,7 @@ CREATE TABLE 02354_annoy_cosine ( id Int32, embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy(100, 'cosineDistance') GRANULARITY 1 + INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 ) ENGINE = MergeTree ORDER BY id @@ -141,72 +141,3 @@ ORDER BY cosineDistance(embedding, [0.0, 0.0, 10.0]) LIMIT 3; DROP TABLE IF EXISTS 02354_annoy_cosine; " | grep "annoy_index" - -# # Check that weird base columns are rejected -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy; - --- Index spans >1 column - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(Float32), - INDEX annoy_index (embedding, id) TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 7 } - --- Index must be created on Array(Float32) or Tuple(Float32) - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Float32, - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(Float64), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Tuple(Float32, Float64), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(LowCardinality(Float32)), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 } - -CREATE TABLE 02354_annoy -( - id Int32, - embedding Array(Nullable(Float32)), - INDEX annoy_index embedding TYPE annoy(100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; -- {serverError 44 }" diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference new file mode 100644 index 00000000000..2d162500f67 --- /dev/null +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -0,0 +1 @@ +Negative tests diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql new file mode 100644 index 00000000000..8df9af1ee73 --- /dev/null +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -0,0 +1,26 @@ +-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check + +SET allow_experimental_annoy_index = 1; + +DROP TABLE IF EXISTS tab; + +SELECT 'Negative tests'; + +-- must have at most 2 arguments +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- first argument must be UInt64 +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- 2nd argument must be String +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- must be created on single column +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index (embedding, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } + +-- must be created on Array/Tuple(Float32) columns +SET allow_suspicious_low_cardinality_types = 1; +CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } From 03b6856556c67fe1c0f7c1df0b28b19556fd3fcc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:39:57 +0000 Subject: [PATCH 049/159] Cosmetics: distance_name --> distance_function --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 41 +++++++++---------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 8 ++-- 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index dc353c97143..a9679453655 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -205,31 +205,31 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const IndexDescription & /*index*/, const SelectQueryInfo & query, - const String& distance_name_, + const String & distance_function_, ContextPtr context) : condition(query, context) - , distance_name(distance_name_) + , distance_function(distance_function_) {} -bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /* idx_granule */) const +bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes"); } bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const { - return condition.alwaysUnknownOrTrue(distance_name); + return condition.alwaysUnknownOrTrue(distance_function); } std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { - if (distance_name == "L2Distance") + if (distance_function == "L2Distance") return getUsefulRangesImpl(idx_granule); - else if (distance_name == "cosineDistance") + else if (distance_function == "cosineDistance") return getUsefulRangesImpl(idx_granule); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } @@ -295,49 +295,48 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI return result_vector; } -MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_) +MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_) : IMergeTreeIndex(index_) , trees(trees_) - , distance_name(distance_name_) + , distance_function(distance_function_) {} MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { - if (distance_name == "L2Distance") + if (distance_function == "L2Distance") return std::make_shared>(index.name, index.sample_block); - else if (distance_name == "cosineDistance") + else if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 - if (distance_name == "L2Distance") + if (distance_function == "L2Distance") return std::make_shared>(index.name, index.sample_block, trees); - if (distance_name == "cosineDistance") + if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block, trees); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_name); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } -MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const +MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const { - return std::make_shared(index, query, distance_name, context); + return std::make_shared(index, query, distance_function, context); }; MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { uint64_t trees = 100; - String distance_name = "L2Distance"; + String distance_function = "L2Distance"; if (!index.arguments.empty()) - distance_name = index.arguments[0].get(); + distance_function = index.arguments[0].get(); if (index.arguments.size() > 1) trees = index.arguments[1].get(); - return std::make_shared(index, trees, distance_name); + return std::make_shared(index, trees, distance_function); } void annoyIndexValidator(const IndexDescription & index, bool /* attach */) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index cde61af2891..9741412e3fa 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -66,7 +66,7 @@ public: MergeTreeIndexConditionAnnoy( const IndexDescription & index, const SelectQueryInfo & query, - const String& distance_name, + const String& distance_function, ContextPtr context); bool alwaysUnknownOrTrue() const override; @@ -82,7 +82,7 @@ private: std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; const ApproximateNearestNeighbour::ANNCondition condition; - const String distance_name; + const String distance_function; }; @@ -90,7 +90,7 @@ class MergeTreeIndexAnnoy : public IMergeTreeIndex { public: - MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String& distance_name_); + MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_); ~MergeTreeIndexAnnoy() override = default; @@ -103,7 +103,7 @@ public: private: const uint64_t trees; - const String distance_name; + const String distance_function; }; From 18304f5aeff627b47965ffd5c07ae6c5d61bface Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:51:21 +0000 Subject: [PATCH 050/159] Check distance function in CREATE TABLE instead of first INSERT --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 13 +++++++++++-- tests/queries/0_stateless/02354_annoy_index.sql | 3 +++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index a9679453655..12446623c30 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -307,7 +307,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const return std::make_shared>(index.name, index.sample_block); else if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); + std::unreachable(); } MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const @@ -317,7 +317,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const return std::make_shared>(index.name, index.sample_block, trees); if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block, trees); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); + std::unreachable(); } MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const @@ -357,6 +357,15 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (index.column_names.size() != 1 || index.data_types.size() != 1) throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "Annoy indexes must be created on a single column"); + /// Check that a supported metric was passed as first argument + + if (!index.arguments.empty()) + { + String distance_name = index.arguments[0].get(); + if (distance_name != "L2Distance" && distance_name != "cosineDistance") + throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index supports only distance functions 'L2Distance' and 'cosineDistance'. Given distance function: {}", distance_name); + } + /// Check data type of indexed column: auto throw_unsupported_underlying_column_exception = [](DataTypePtr data_type) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 8df9af1ee73..3a5fb6817ff 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -24,3 +24,6 @@ CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE a CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } + +-- reject unsupported distance functions +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } From d3158a28770339f36f9c0e69acb6d271ced351f6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 20:56:27 +0000 Subject: [PATCH 051/159] Cosmetics: Consolidate parameters --- src/Core/Settings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6a0833aef60..8055e9b4880 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -719,7 +719,6 @@ class IColumn; \ M(Bool, parallelize_output_from_storages, true, "Parallelize output for reading step from storage. It allows parallelizing query processing right after reading from storage if possible", 0) \ M(String, insert_deduplication_token, "", "If not empty, used for duplicate detection instead of data digest", 0) \ - M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \ M(Bool, count_distinct_optimization, false, "Rewrite count distinct to subquery of group by", 0) \ M(Bool, throw_if_no_data_to_insert, true, "Enables or disables empty INSERTs, enabled by default", 0) \ M(Bool, compatibility_ignore_auto_increment_in_create_table, false, "Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL", 0) \ @@ -742,7 +741,8 @@ class IColumn; M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ - M(UInt64, max_limit_for_ann_queries, 1000000, "Maximum limit value for using ANN indexes is used to prevent memory overflow in search queries for indexes", 0) \ + M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexs.", 0) \ + M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ From a973ac5dbb99e7fc624742b34fc507935cd792e4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:03:50 +0000 Subject: [PATCH 052/159] Replace weird generic ANN setting by Annoy-specific parameter --- .../mergetree-family/annindexes.md | 3 ++- src/Core/Settings.h | 2 +- src/Storages/MergeTree/CommonANNIndexes.cpp | 1 - src/Storages/MergeTree/CommonANNIndexes.h | 7 ------ .../MergeTree/MergeTreeIndexAnnoy.cpp | 18 +++----------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 5 ++-- .../0_stateless/02354_annoy_index.reference | 12 ++++++++++ .../queries/0_stateless/02354_annoy_index.sql | 24 +++++++++++++++++++ 8 files changed, 45 insertions(+), 27 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 03617a1a709..9b4de150235 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -135,7 +135,8 @@ ORDER BY id; Annoy supports `L2Distance` and `cosineDistance`. -In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time trade-off between better accuracy and speed. +Setting `search_k` (default `LIMIT * NumTrees`) determines how many nodes the Annoy index will inspect during SELECT queries. The setting +can be used to balance performance and accuracy at runtime. __Example__: ``` sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8055e9b4880..3e10f48a2fb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -742,7 +742,7 @@ class IColumn; M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexs.", 0) \ - M(String, ann_index_select_query_params, "", "Parameters passed to ANN indexes in SELECT queries, the format is 'param1=x, param2=y, ...'", 0) \ + M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index f0c6f256f73..4748c869f83 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -60,7 +60,6 @@ namespace ApproximateNearestNeighbour ANNCondition::ANNCondition(const SelectQueryInfo & query_info, ContextPtr context) : block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, - ann_index_select_query_params{context->getSettings().get("ann_index_select_query_params").get()}, index_granularity{context->getMergeTreeSettings().get("index_granularity").get()}, limit_restriction{context->getSettings().get("max_limit_for_ann_queries").get()}, index_is_useful{checkQueryStructure(query_info)} {} diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index fefb9584863..4253bce703a 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -82,8 +82,6 @@ struct ANNQueryInformation * spaceDimension(which is targetVector's components count) * column * objects count from LIMIT clause(for both queries) - * settings str, if query has settings section with new 'ann_index_select_query_params' value, - than you can get the new value(empty by default) calling method getSettingsStr * queryHasOrderByClause and queryHasWhereClause return true if query matches the type Search query type is also recognized for PREWHERE clause @@ -121,11 +119,7 @@ public: // length's value from LIMIT clause UInt64 getLimit() const; - // value of 'ann_index_select_query_params' if have in SETTINGS clause, empty string otherwise - String getParamsStr() const { return ann_index_select_query_params; } - private: - struct RPNElement { enum Function @@ -217,7 +211,6 @@ private: std::optional query_information; // Get from settings ANNIndex parameters - String ann_index_select_query_params; UInt64 index_granularity; /// only queries with a lower limit can be considered to avoid memory overflow UInt64 limit_restriction; diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 12446623c30..6ffb7aecb7f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -209,6 +210,7 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( ContextPtr context) : condition(query, context) , distance_function(distance_function_) + , search_k(context->getSettings().get("annoy_index_search_k_nodes").get()) {} @@ -264,21 +266,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI neighbors.reserve(limit); distances.reserve(limit); - int k_search = -1; - String params_str = condition.getParamsStr(); - if (!params_str.empty()) - { - try - { - /// k_search=... (algorithm will inspect up to search_k nodes which defaults to n_trees * n if not provided) - k_search = std::stoi(params_str.data() + 9); - } - catch (...) - { - throw Exception(ErrorCodes::INCORRECT_QUERY, "Setting of the annoy index should be int"); - } - } - annoy->get_nns_by_vector(target_vec.data(), limit, k_search, &neighbors, &distances); + annoy->get_nns_by_vector(target_vec.data(), limit, static_cast(search_k), &neighbors, &distances); std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 9741412e3fa..fbc6b21fa6b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -69,20 +69,21 @@ public: const String& distance_function, ContextPtr context); + ~MergeTreeIndexConditionAnnoy() override = default; + bool alwaysUnknownOrTrue() const override; bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const override; - ~MergeTreeIndexConditionAnnoy() override = default; - private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; const ApproximateNearestNeighbour::ANNCondition condition; const String distance_function; + const Int64 search_k; }; diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 2d162500f67..7da442cb905 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -1 +1,13 @@ +parameter annoy_index_search_k_nodes +parameter max_limit_for_ann_queries +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 Negative tests diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 3a5fb6817ff..3590b7d316e 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -4,6 +4,30 @@ SET allow_experimental_annoy_index = 1; DROP TABLE IF EXISTS tab; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; +-- SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +SELECT 'parameter annoy_index_search_k_nodes'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1]) +LIMIT 5 +SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results + +SELECT 'parameter max_limit_for_ann_queries'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [5.3, 7.3, 2.1]) +LIMIT 5 +SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index + +DROP TABLE tab; + +DROP TABLE IF EXISTS tab; + SELECT 'Negative tests'; -- must have at most 2 arguments From 6d3431d2ff0325282ea373bfeafed6a2e1946577 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:12:40 +0000 Subject: [PATCH 053/159] Cosmetics: Sort includes --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 6ffb7aecb7f..fe3ea322b91 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -2,15 +2,15 @@ #include +#include #include #include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include namespace DB From f800940639bbe490e619594db62a83f7f8f2f80d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:13:33 +0000 Subject: [PATCH 054/159] Cosmetics: Shuffle statements --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 16b27c2c820..aa340d6afc1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1686,7 +1686,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( { if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin) granule = reader.read(); - const auto * gin_filter_condition = dynamic_cast(&*condition); // Cast to Ann condition auto ann_condition = std::dynamic_pointer_cast(condition); if (ann_condition != nullptr) @@ -1714,6 +1713,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( } bool result = false; + const auto * gin_filter_condition = dynamic_cast(&*condition); if (!gin_filter_condition) result = condition->mayBeTrueOnGranule(granule); else From 660760782ad6357c34f55113f3a9522e94b30dd3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:16:03 +0000 Subject: [PATCH 055/159] Rewrite ANN docs --- .../mergetree-family/annindexes.md | 140 ++++++++++-------- 1 file changed, 78 insertions(+), 62 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 9b4de150235..0cc1cff2dad 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -1,78 +1,89 @@ # Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex} -The main task that indexes achieve is to quickly find nearest neighbors for multidimensional data. An example of such a problem can be finding similar pictures (texts) for a given picture (text). That problem can be reduced to finding the nearest [embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning). They can be created from data using [UDF](/docs/en/sql-reference/functions/index.md/#executable-user-defined-functions). +Nearest neighborhood search refers to the problem of finding the point(s) with the smallest distance to a given point in an n-dimensional +space. Since exact search is in practice usually typically too slow, the task is often solved with approximate algorithms. A popular use +case of of neighbor search is finding similar pictures (texts) for a given picture (text). Pictures (texts) can be decomposed into +[embeddings](https://cloud.google.com/architecture/overview-extracting-and-serving-feature-embeddings-for-machine-learning), and instead of +comparing pictures (texts) pixel-by-pixel (character-by-character), only the embeddings are compared. -The next queries find the closest neighbors in N-dimensional space using the L2 (Euclidean) distance: -``` sql -SELECT * -FROM table_name -WHERE L2Distance(Column, Point) < MaxDistance +In terms of SQL, the problem can be expressed as follows: + +``` sql +SELECT * +FROM table +WHERE L2Distance(column, Point) < MaxDistance LIMIT N ``` -``` sql -SELECT * -FROM table_name -ORDER BY L2Distance(Column, Point) +``` sql +SELECT * +FROM table +ORDER BY L2Distance(column, Point) LIMIT N ``` -But it will take some time for execution because of the long calculation of the distance between `TargetEmbedding` and all other vectors. This is where ANN indexes can help. They store a compact approximation of the search space (e.g. using clustering, search trees, etc.) and are able to compute approximate neighbors quickly. + +The queries are expensive because the L2 distance (Euclidean distance) between all points in `column` and `Point` must be computed. To speed this process up, ANN indexes store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly. ## Indexes Structure -Approximate Nearest Neighbor Search Indexes (`ANNIndexes`) are similar to skip indexes. They are constructed by some granules and determine which of them should be skipped. Compared to skip indices, ANN indices use their results not only to skip some group of granules, but also to select particular granules from a set of granules. +Approximate Nearest Neighbor Search Indexes (or `ANNIndexes`) are similar to skip indexes. They are constructed over granules and determine which granules can be skipped. Compared to skip indices, ANN indices are not only able to skip granules, they can also to select particular granules from a set of granules. -`ANNIndexes` are designed to speed up two types of queries: +`ANNIndexes` support two types of queries: -- ###### Type 1: Where - ``` sql - SELECT * - FROM table_name - WHERE DistanceFunction(Column, Point) < MaxDistance +- WHERE queries: + ``` sql + SELECT * + FROM table + WHERE DistanceFunction(column, Point) < MaxDistance LIMIT N ``` -- ###### Type 2: Order by + +- ORDER BY queries: ``` sql - SELECT * - FROM table_name [WHERE ...] - ORDER BY DistanceFunction(Column, Point) + SELECT * + FROM table [WHERE ...] + ORDER BY DistanceFunction(column, Point) LIMIT N ``` -In these queries, `DistanceFunction` is selected from [distance functions](/docs/en/sql-reference/functions/distance-functions.md). `Point` is a known vector (something like `(0.1, 0.1, ... )`). To avoid writing large vectors, use [client parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters). `Value` - a float value that will bound the neighbourhood. +`DistanceFunction` is a [distance functions](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a given vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a float value which restricts the size of the neighbourhood. -:::note -ANN index can't speed up query that satisfies both types (`where + order by`, only one of them). All queries must have the limit, as algorithms are used to find nearest neighbors and need a specific number of them. -::: +To avoid writing large vectors, you can also use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. -:::note -Indexes are applied only to queries with a limit less than the `max_limit_for_ann_queries` setting. This helps to avoid memory overflows in queries with a large limit. `max_limit_for_ann_queries` setting can be changed if you know you can provide enough memory. The default value is `1000000`. -::: +```bash +clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(embedding, {vec: Array(Float32)}) < 1.0" +``` -Both types of queries are handled the same way. The indexes get `n` neighbors (where `n` is taken from the `LIMIT` clause) and work with them. In `ORDER BY` query they remember the numbers of all parts of the granule that have at least one of neighbor. In `WHERE` query they remember only those parts that satisfy the requirements. +ANN index cannot speed up query that contain both `WHERE` and `ORDER BY`. Queries must have a limit, as the approximate algorithms used to determine the nearest neighbors require a specific number of them. + +Indexes are only used for queries with a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This helps to prevent memory overflows in queries with a large limit. + +Both types of queries are processed similarly. The indexes are passed the number of neighbors `N`. In `ORDER BY` query they remember the numbers of all parts of the granule that have at least one of neighbor. In `WHERE` query they remember only those parts that satisfy the requirements. -## Create table with ANNIndex +## Creating Tables with an ANN Index -This feature is disabled by default. To enable it, set `allow_experimental_annoy_index` to 1. Also, this feature is disabled on ARM, due to likely problems with the algorithm. +As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`. + +Syntax: ```sql -CREATE TABLE t +CREATE TABLE table ( `id` Int64, - `data` Tuple(Float32, Float32, Float32), - INDEX ann_index_name data TYPE ann_index_type(ann_index_parameters) GRANULARITY N + `embedding` Tuple(Float32, Float32, Float32), + INDEX embedding TYPE () GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` ```sql -CREATE TABLE t +CREATE TABLE table ( `id` Int64, - `data` Array(Float32), - INDEX ann_index_name data TYPE ann_index_type(ann_index_parameters) GRANULARITY N + `embedding` Array(Float32), + INDEX embedding TYPE () GRANULARITY N ) ENGINE = MergeTree ORDER BY id; @@ -80,69 +91,74 @@ ORDER BY id; With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. -As the indexes are built only during insertions into table, `INSERT` and `OPTIMIZE` queries are slower than for ordinary table. At this stage indexes remember all the information about the given data. ANNIndexes should be used if you have immutable or rarely changed data and many read requests. - -You can create your table with index which uses certain algorithm. Now only indices based on the following algorithms are supported: +Note that ANN indexes are built during column insertion and merge, i.e. `INSERT` and `OPTIMIZE` statements are slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changing data in conjunction with many read requests. # Index list + - [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) # Annoy {#annoy} -Implementation of the algorithm was taken from [this repository](https://github.com/spotify/annoy). + +(currently disabled on ARM due to problems with the algorithm) + +This ANN index type implements [Annoy indexes](https://github.com/spotify/annoy). Short description of the algorithm: The algorithm recursively divides in half all space by random linear surfaces (lines in 2D, planes in 3D etc.). Thus it makes tree of polyhedrons and points that they contains. Repeating the operation several times for greater accuracy it creates a forest. To find K Nearest Neighbours it goes down through the trees and fills the buffer of closest points using the priority queue of polyhedrons. Next, it sorts buffer and return the nearest K points. -__Examples__: +Examples: + ```sql -CREATE TABLE t +CREATE TABLE table ( id Int64, - data Tuple(Float32, Float32, Float32), - INDEX ann_index_name data TYPE annoy(NumTrees, DistanceName) GRANULARITY N + embedding Tuple(Float32, Float32, Float32), + INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` ```sql -CREATE TABLE t +CREATE TABLE table ( id Int64, - data Array(Float32), - INDEX ann_index_name data TYPE annoy(NumTrees, DistanceName) GRANULARITY N + embedding Array(Float32), + INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` :::note -Table with array field will work faster, but all arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(data) = 256`. +Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(embedding) = 256`. ::: -Parameter `NumTrees` is the number of trees which the algorithm will create. The bigger it is, the slower (approximately linear) it works (in both `CREATE` and `SELECT` requests), but the better accuracy you get (adjusted for randomness). By default it is set to `100`. Parameter `DistanceName` is name of distance function. By default it is set to `L2Distance`. It can be set without changing first parameter, for example +Parameter `DistanceName` is name of a distance function with default `L2Distance`. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results. + ```sql -CREATE TABLE t +CREATE TABLE table ( id Int64, - data Array(Float32), - INDEX ann_index_name data TYPE annoy('cosineDistance') GRANULARITY N + embedding Array(Float32), + INDEX ann_index_name embedding TYPE annoy('cosineDistance') GRANULARITY N ) ENGINE = MergeTree ORDER BY id; ``` -Annoy supports `L2Distance` and `cosineDistance`. +Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. -Setting `search_k` (default `LIMIT * NumTrees`) determines how many nodes the Annoy index will inspect during SELECT queries. The setting -can be used to balance performance and accuracy at runtime. +Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many nodes are inspected during SELECTs. It can be used to +balance runtime and accuracy at runtime. + +Example: -__Example__: ``` sql -SELECT * -FROM table_name [WHERE ...] -ORDER BY L2Distance(Column, Point) +SELECT * +FROM table_name [WHERE ...] +ORDER BY L2Distance(column, Point) LIMIT N -SETTING ann_index_select_query_params=`k_search=100` +SETTINGS annoy_index_search_k_nodes=100 ``` From 662b0fb822e6e297c1e490fef8e2c09e22f4af09 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:31:33 +0000 Subject: [PATCH 056/159] Cosmetics: Unwrap common ANN code from nested namespace --- src/Storages/MergeTree/CommonANNIndexes.cpp | 75 +++++++++---------- src/Storages/MergeTree/CommonANNIndexes.h | 27 +++---- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeIndexAnnoy.cpp | 3 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 +- 5 files changed, 50 insertions(+), 61 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 4748c869f83..5c42774fb24 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -24,10 +24,8 @@ namespace ErrorCodes namespace { -namespace ANN = ApproximateNearestNeighbour; - template -void extractTargetVectorFromLiteral(ANN::ANNQueryInformation::Embedding & target, Literal literal) +void extractTargetVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & target, Literal literal) { Float64 float_element_of_target_vector; Int64 int_element_of_target_vector; @@ -43,28 +41,25 @@ void extractTargetVectorFromLiteral(ANN::ANNQueryInformation::Embedding & target } } -ANN::ANNQueryInformation::Metric castMetricFromStringToType(String metric_name) +ApproximateNearestNeighborInformation::Metric castMetricFromStringToType(String metric_name) { if (metric_name == "L2Distance") - return ANN::ANNQueryInformation::Metric::L2; + return ApproximateNearestNeighborInformation::Metric::L2; if (metric_name == "LpDistance") - return ANN::ANNQueryInformation::Metric::Lp; - return ANN::ANNQueryInformation::Metric::Unknown; + return ApproximateNearestNeighborInformation::Metric::Lp; + return ApproximateNearestNeighborInformation::Metric::Unknown; } } -namespace ApproximateNearestNeighbour -{ - -ANNCondition::ANNCondition(const SelectQueryInfo & query_info, +ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) : block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, index_granularity{context->getMergeTreeSettings().get("index_granularity").get()}, limit_restriction{context->getSettings().get("max_limit_for_ann_queries").get()}, index_is_useful{checkQueryStructure(query_info)} {} -bool ANNCondition::alwaysUnknownOrTrue(String metric_name) const +bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric_name) const { if (!index_is_useful) return true; // Query isn't supported @@ -72,64 +67,64 @@ bool ANNCondition::alwaysUnknownOrTrue(String metric_name) const return !(castMetricFromStringToType(metric_name) == query_information->metric); } -float ANNCondition::getComparisonDistanceForWhereQuery() const +float ApproximateNearestNeighborCondition::getComparisonDistanceForWhereQuery() const { if (index_is_useful && query_information.has_value() - && query_information->query_type == ANNQueryInformation::Type::Where) + && query_information->query_type == ApproximateNearestNeighborInformation::Type::Where) return query_information->distance; throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported method for this query type"); } -UInt64 ANNCondition::getLimit() const +UInt64 ApproximateNearestNeighborCondition::getLimit() const { if (index_is_useful && query_information.has_value()) return query_information->limit; throw Exception(ErrorCodes::LOGICAL_ERROR, "No LIMIT section in query, not supported"); } -std::vector ANNCondition::getTargetVector() const +std::vector ApproximateNearestNeighborCondition::getTargetVector() const { if (index_is_useful && query_information.has_value()) return query_information->target; throw Exception(ErrorCodes::LOGICAL_ERROR, "Target vector was requested for useless or uninitialized index."); } -size_t ANNCondition::getNumOfDimensions() const +size_t ApproximateNearestNeighborCondition::getNumOfDimensions() const { if (index_is_useful && query_information.has_value()) return query_information->target.size(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of dimensions was requested for useless or uninitialized index."); } -String ANNCondition::getColumnName() const +String ApproximateNearestNeighborCondition::getColumnName() const { if (index_is_useful && query_information.has_value()) return query_information->column_name; throw Exception(ErrorCodes::LOGICAL_ERROR, "Column name was requested for useless or uninitialized index."); } -ANNQueryInformation::Metric ANNCondition::getMetricType() const +ApproximateNearestNeighborInformation::Metric ApproximateNearestNeighborCondition::getMetricType() const { if (index_is_useful && query_information.has_value()) return query_information->metric; throw Exception(ErrorCodes::LOGICAL_ERROR, "Metric name was requested for useless or uninitialized index."); } -float ANNCondition::getPValueForLpDistance() const +float ApproximateNearestNeighborCondition::getPValueForLpDistance() const { if (index_is_useful && query_information.has_value()) return query_information->p_for_lp_dist; throw Exception(ErrorCodes::LOGICAL_ERROR, "P from LPDistance was requested for useless or uninitialized index."); } -ANNQueryInformation::Type ANNCondition::getQueryType() const +ApproximateNearestNeighborInformation::Type ApproximateNearestNeighborCondition::getQueryType() const { if (index_is_useful && query_information.has_value()) return query_information->query_type; throw Exception(ErrorCodes::LOGICAL_ERROR, "Query type was requested for useless or uninitialized index."); } -bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) +bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryInfo & query) { // RPN-s for different sections of the query RPN rpn_prewhere_clause; @@ -138,9 +133,9 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) RPNElement rpn_limit; UInt64 limit; - ANNQueryInformation prewhere_info; - ANNQueryInformation where_info; - ANNQueryInformation order_by_info; + ApproximateNearestNeighborInformation prewhere_info; + ApproximateNearestNeighborInformation where_info; + ApproximateNearestNeighborInformation order_by_info; // Build rpns for query sections const auto & select = query.query->as(); @@ -195,7 +190,7 @@ bool ANNCondition::checkQueryStructure(const SelectQueryInfo & query) return query_information.has_value(); } -void ANNCondition::traverseAST(const ASTPtr & node, RPN & rpn) +void ApproximateNearestNeighborCondition::traverseAST(const ASTPtr & node, RPN & rpn) { // If the node is ASTFunction, it may have children nodes if (const auto * func = node->as()) @@ -214,7 +209,7 @@ void ANNCondition::traverseAST(const ASTPtr & node, RPN & rpn) rpn.emplace_back(std::move(element)); } -bool ANNCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) +bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) { // Match Functions if (const auto * function = node->as()) @@ -259,7 +254,7 @@ bool ANNCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) return tryCastToConstType(node, out); } -bool ANNCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) +bool ApproximateNearestNeighborCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) { Field const_value; DataTypePtr const_type; @@ -318,18 +313,18 @@ bool ANNCondition::tryCastToConstType(const ASTPtr & node, RPNElement & out) return false; } -void ANNCondition::traverseOrderByAST(const ASTPtr & node, RPN & rpn) +void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node, RPN & rpn) { if (const auto * expr_list = node->as()) if (const auto * order_by_element = expr_list->children.front()->as()) traverseAST(order_by_element->children.front(), rpn); } -// Returns true and stores ANNQueryInformation if the query has valid WHERE clause -bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) +// Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause +bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr) { /// Fill query type field - expr.query_type = ANNQueryInformation::Type::Where; + expr.query_type = ApproximateNearestNeighborInformation::Type::Where; // WHERE section must have at least 5 expressions // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) @@ -381,10 +376,10 @@ bool ANNCondition::matchRPNWhere(RPN & rpn, ANNQueryInformation & expr) } // Returns true and stores ANNExpr if the query has valid ORDERBY clause -bool ANNCondition::matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr) +bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr) { /// Fill query type field - expr.query_type = ANNQueryInformation::Type::OrderBy; + expr.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) @@ -393,11 +388,11 @@ bool ANNCondition::matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr) auto iter = rpn.begin(); auto end = rpn.end(); - return ANNCondition::matchMainParts(iter, end, expr); + return ApproximateNearestNeighborCondition::matchMainParts(iter, end, expr); } // Returns true and stores Length if we have valid LIMIT clause in query -bool ANNCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) +bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) { if (rpn.function == RPNElement::FUNCTION_INT_LITERAL) { @@ -409,7 +404,7 @@ bool ANNCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) } /* Matches dist function, target vector, column name */ -bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ANNQueryInformation & expr) +bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr) { bool identifier_found = false; @@ -420,7 +415,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en expr.metric = castMetricFromStringToType(iter->func_name); ++iter; - if (expr.metric == ANN::ANNQueryInformation::Metric::Lp) + if (expr.metric == ApproximateNearestNeighborInformation::Metric::Lp) { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL && iter->function != RPNElement::FUNCTION_INT_LITERAL) @@ -497,7 +492,7 @@ bool ANNCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & en } // Gets float or int from AST node -float ANNCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) +float ApproximateNearestNeighborCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) { if (iter->float_literal.has_value()) return iter->float_literal.value(); @@ -507,5 +502,3 @@ float ANNCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) } } - -} diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 4253bce703a..37695586515 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -9,9 +9,6 @@ namespace DB { -namespace ApproximateNearestNeighbour -{ - /** * Queries for Approximate Nearest Neighbour Search * have similar structure: @@ -25,7 +22,7 @@ namespace ApproximateNearestNeighbour * 1) p for LpDistance function * 2) distance to compare with (only for where queries) */ -struct ANNQueryInformation +struct ApproximateNearestNeighborInformation { using Embedding = std::vector; @@ -51,7 +48,7 @@ struct ANNQueryInformation }; /** - Class ANNCondition, is responsible for recognizing special query types which + Class ApproximateNearestNeighborCondition, is responsible for recognizing special query types which can be speeded up by ANN Indexes. It parses the SQL query and checks if it matches ANNIndexes. The recognizing method - alwaysUnknownOrTrue returns false if we can speed up the query, and true otherwise. @@ -87,10 +84,10 @@ struct ANNQueryInformation Search query type is also recognized for PREWHERE clause */ -class ANNCondition +class ApproximateNearestNeighborCondition { public: - ANNCondition(const SelectQueryInfo & query_info, + ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context); // false if query can be speeded up, true otherwise @@ -107,12 +104,12 @@ public: String getColumnName() const; - ANNQueryInformation::Metric getMetricType() const; + ApproximateNearestNeighborInformation::Metric getMetricType() const; // the P- value if the metric is 'LpDistance' float getPValueForLpDistance() const; - ANNQueryInformation::Type getQueryType() const; + ApproximateNearestNeighborInformation::Type getQueryType() const; UInt64 getIndexGranularity() const { return index_granularity; } @@ -191,16 +188,16 @@ private: void traverseOrderByAST(const ASTPtr & node, RPN & rpn); // Returns true and stores ANNExpr if the query has valid WHERE section - static bool matchRPNWhere(RPN & rpn, ANNQueryInformation & expr); + static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr); // Returns true and stores ANNExpr if the query has valid ORDERBY section - static bool matchRPNOrderBy(RPN & rpn, ANNQueryInformation & expr); + static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr); // Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); /* Matches dist function, target vector, column name */ - static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ANNQueryInformation & expr); + static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr); // Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); @@ -208,7 +205,7 @@ private: Block block_with_constants; // true if we have one of two supported query types - std::optional query_information; + std::optional query_information; // Get from settings ANNIndex parameters UInt64 index_granularity; @@ -218,12 +215,10 @@ private: }; // condition interface for Ann indexes. Returns vector of indexes of ranges in granule which are useful for query. -class IMergeTreeIndexConditionAnn : public IMergeTreeIndexCondition +class IMergeTreeIndexConditionApproximateNearestNeighbor : public IMergeTreeIndexCondition { public: virtual std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const = 0; }; } - -} diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index aa340d6afc1..3d290ea12ac 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1687,7 +1687,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( if (index_mark != index_range.begin || !granule || last_index_mark != index_range.begin) granule = reader.read(); // Cast to Ann condition - auto ann_condition = std::dynamic_pointer_cast(condition); + auto ann_condition = std::dynamic_pointer_cast(condition); if (ann_condition != nullptr) { // vector of indexes of useful ranges diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index fe3ea322b91..190f76fba5e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -240,7 +240,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI { UInt64 limit = condition.getLimit(); UInt64 index_granularity = condition.getIndexGranularity(); - std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighbour::ANNQueryInformation::Type::Where + std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where ? std::optional(condition.getComparisonDistanceForWhereQuery()) : std::nullopt; @@ -267,6 +267,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI distances.reserve(limit); annoy->get_nns_by_vector(target_vec.data(), limit, static_cast(search_k), &neighbors, &distances); + std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index fbc6b21fa6b..95041ea31fb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -60,7 +60,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator }; -class MergeTreeIndexConditionAnnoy final : public ApproximateNearestNeighbour::IMergeTreeIndexConditionAnn +class MergeTreeIndexConditionAnnoy final : public IMergeTreeIndexConditionApproximateNearestNeighbor { public: MergeTreeIndexConditionAnnoy( @@ -81,7 +81,7 @@ private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; - const ApproximateNearestNeighbour::ANNCondition condition; + const ApproximateNearestNeighborCondition condition; const String distance_function; const Int64 search_k; }; From 999e4c33065279f4337387ece0343da36ca03098 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:37:20 +0000 Subject: [PATCH 057/159] Cosmetics: Less generic variable naming --- src/Storages/MergeTree/CommonANNIndexes.cpp | 52 ++++++++++----------- src/Storages/MergeTree/CommonANNIndexes.h | 6 +-- 2 files changed, 29 insertions(+), 29 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 5c42774fb24..669055068b8 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -321,10 +321,10 @@ void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node } // Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause -bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr) +bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - expr.query_type = ApproximateNearestNeighborInformation::Type::Where; + ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; // WHERE section must have at least 5 expressions // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) @@ -347,9 +347,9 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL) return false; - expr.distance = getFloatOrIntLiteralOrPanic(iter); - if (expr.distance < 0) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", expr.distance); + ann_info.distance = getFloatOrIntLiteralOrPanic(iter); + if (ann_info.distance < 0) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", ann_info.distance); ++iter; @@ -358,17 +358,17 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe return false; auto end = rpn.end(); - if (!matchMainParts(iter, end, expr)) + if (!matchMainParts(iter, end, ann_info)) return false; if (greater_case) { - if (expr.target.size() < 2) + if (ann_info.target.size() < 2) return false; - expr.distance = expr.target.back(); - if (expr.distance < 0) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", expr.distance); - expr.target.pop_back(); + ann_info.distance = ann_info.target.back(); + if (ann_info.distance < 0) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", ann_info.distance); + ann_info.target.pop_back(); } // query is ok @@ -376,10 +376,10 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe } // Returns true and stores ANNExpr if the query has valid ORDERBY clause -bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr) +bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - expr.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; + ann_info.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) @@ -388,7 +388,7 @@ bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, Approximate auto iter = rpn.begin(); auto end = rpn.end(); - return ApproximateNearestNeighborCondition::matchMainParts(iter, end, expr); + return ApproximateNearestNeighborCondition::matchMainParts(iter, end, ann_info); } // Returns true and stores Length if we have valid LIMIT clause in query @@ -404,7 +404,7 @@ bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 } /* Matches dist function, target vector, column name */ -bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr) +bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info) { bool identifier_found = false; @@ -412,22 +412,22 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; - expr.metric = castMetricFromStringToType(iter->func_name); + ann_info.metric = castMetricFromStringToType(iter->func_name); ++iter; - if (expr.metric == ApproximateNearestNeighborInformation::Metric::Lp) + if (ann_info.metric == ApproximateNearestNeighborInformation::Metric::Lp) { if (iter->function != RPNElement::FUNCTION_FLOAT_LITERAL && iter->function != RPNElement::FUNCTION_INT_LITERAL) return false; - expr.p_for_lp_dist = getFloatOrIntLiteralOrPanic(iter); + ann_info.p_for_lp_dist = getFloatOrIntLiteralOrPanic(iter); ++iter; } if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { identifier_found = true; - expr.column_name = std::move(iter->identifier.value()); + ann_info.column_name = std::move(iter->identifier.value()); ++iter; } @@ -436,13 +436,13 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(expr.target, iter->tuple_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); ++iter; } if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(expr.target, iter->array_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); ++iter; } @@ -457,12 +457,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(expr.target, iter->tuple_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); ++iter; } else if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(expr.target, iter->array_literal); + extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); ++iter; } else @@ -473,12 +473,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c { if (iter->function == RPNElement::FUNCTION_FLOAT_LITERAL || iter->function == RPNElement::FUNCTION_INT_LITERAL) - expr.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); + ann_info.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); else if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { if (identifier_found) return false; - expr.column_name = std::move(iter->identifier.value()); + ann_info.column_name = std::move(iter->identifier.value()); identifier_found = true; } else @@ -488,7 +488,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c } // Final checks of correctness - return identifier_found && !expr.target.empty(); + return identifier_found && !ann_info.target.empty(); } // Gets float or int from AST node diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 37695586515..0b207585048 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -188,16 +188,16 @@ private: void traverseOrderByAST(const ASTPtr & node, RPN & rpn); // Returns true and stores ANNExpr if the query has valid WHERE section - static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & expr); + static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); // Returns true and stores ANNExpr if the query has valid ORDERBY section - static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & expr); + static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); // Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); /* Matches dist function, target vector, column name */ - static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & expr); + static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info); // Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); From 567d54a26848ba0f5bf4ad38d226def02a92280b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:38:00 +0000 Subject: [PATCH 058/159] Cosmetics: more constness --- src/Storages/MergeTree/CommonANNIndexes.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 0b207585048..68ed217ef45 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -208,9 +208,9 @@ private: std::optional query_information; // Get from settings ANNIndex parameters - UInt64 index_granularity; + const UInt64 index_granularity; /// only queries with a lower limit can be considered to avoid memory overflow - UInt64 limit_restriction; + const UInt64 limit_restriction; bool index_is_useful = false; }; From 6fe208832d6c3ba8340fd73d5728efad567188ba Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:42:46 +0000 Subject: [PATCH 059/159] Cosmetics: target vector --> reference vector --- src/Storages/MergeTree/CommonANNIndexes.cpp | 48 +++++++++---------- src/Storages/MergeTree/CommonANNIndexes.h | 22 ++++----- .../MergeTree/MergeTreeIndexAnnoy.cpp | 6 +-- 3 files changed, 38 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 669055068b8..20707a148ae 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -25,19 +25,19 @@ namespace { template -void extractTargetVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & target, Literal literal) +void extraceReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & reference_vector, Literal literal) { - Float64 float_element_of_target_vector; - Int64 int_element_of_target_vector; + Float64 float_element_of_reference_vector; + Int64 int_element_of_reference_vector; for (const auto & value : literal.value()) { - if (value.tryGet(float_element_of_target_vector)) - target.emplace_back(float_element_of_target_vector); - else if (value.tryGet(int_element_of_target_vector)) - target.emplace_back(static_cast(int_element_of_target_vector)); + if (value.tryGet(float_element_of_reference_vector)) + reference_vector.emplace_back(float_element_of_reference_vector); + else if (value.tryGet(int_element_of_reference_vector)) + reference_vector.emplace_back(static_cast(int_element_of_reference_vector)); else - throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type of elements in target vector. Only float or int are supported."); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Wrong type of elements in reference vector. Only float or int are supported."); } } @@ -82,17 +82,17 @@ UInt64 ApproximateNearestNeighborCondition::getLimit() const throw Exception(ErrorCodes::LOGICAL_ERROR, "No LIMIT section in query, not supported"); } -std::vector ApproximateNearestNeighborCondition::getTargetVector() const +std::vector ApproximateNearestNeighborCondition::getReferenceVector() const { if (index_is_useful && query_information.has_value()) - return query_information->target; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Target vector was requested for useless or uninitialized index."); + return query_information->reference_vector; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Reference vector was requested for useless or uninitialized index."); } size_t ApproximateNearestNeighborCondition::getNumOfDimensions() const { if (index_is_useful && query_information.has_value()) - return query_information->target.size(); + return query_information->reference_vector.size(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of dimensions was requested for useless or uninitialized index."); } @@ -327,7 +327,7 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; // WHERE section must have at least 5 expressions - // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(TargetVector(floats)) + // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) if (rpn.size() < 5) return false; @@ -363,12 +363,12 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe if (greater_case) { - if (ann_info.target.size() < 2) + if (ann_info.reference_vector.size() < 2) return false; - ann_info.distance = ann_info.target.back(); + ann_info.distance = ann_info.reference_vector.back(); if (ann_info.distance < 0) throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance can't be negative. Got {}", ann_info.distance); - ann_info.target.pop_back(); + ann_info.reference_vector.pop_back(); } // query is ok @@ -403,12 +403,12 @@ bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 return false; } -/* Matches dist function, target vector, column name */ +/* Matches dist function, referencer vector, column name */ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info) { bool identifier_found = false; - // Matches DistanceFunc->[Column]->[Tuple(array)Func]->TargetVector(floats)->[Column] + // Matches DistanceFunc->[Column]->[Tuple(array)Func]->ReferenceVector(floats)->[Column] if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; @@ -436,13 +436,13 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } @@ -457,12 +457,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extractTargetVectorFromLiteral(ann_info.target, iter->tuple_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } else if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extractTargetVectorFromLiteral(ann_info.target, iter->array_literal); + extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } else @@ -473,7 +473,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c { if (iter->function == RPNElement::FUNCTION_FLOAT_LITERAL || iter->function == RPNElement::FUNCTION_INT_LITERAL) - ann_info.target.emplace_back(getFloatOrIntLiteralOrPanic(iter)); + ann_info.reference_vector.emplace_back(getFloatOrIntLiteralOrPanic(iter)); else if (iter->function == RPNElement::FUNCTION_IDENTIFIER) { if (identifier_found) @@ -488,7 +488,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c } // Final checks of correctness - return identifier_found && !ann_info.target.empty(); + return identifier_found && !ann_info.reference_vector.empty(); } // Gets float or int from AST node diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 68ed217ef45..9c075139707 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -12,7 +12,7 @@ namespace DB /** * Queries for Approximate Nearest Neighbour Search * have similar structure: - * 1) target vector from which all distances are calculated + * 1) reference vector from which all distances are calculated * 2) metric name (e.g L2Distance, LpDistance, etc.) * 3) name of column with embeddings * 4) type of query @@ -27,7 +27,7 @@ struct ApproximateNearestNeighborInformation using Embedding = std::vector; // Extracted data from valid query - Embedding target; + Embedding reference_vector; enum class Metric { Unknown, @@ -56,14 +56,14 @@ struct ApproximateNearestNeighborInformation There are two main patterns of queries being supported 1) Search query type - SELECT * FROM * WHERE DistanceFunc(column, target_vector) < floatLiteral LIMIT count + SELECT * FROM * WHERE DistanceFunc(column, reference) < floatLiteral LIMIT count 2) OrderBy query type - SELECT * FROM * WHERE * ORDERBY DistanceFunc(column, target_vector) LIMIT count + SELECT * FROM * WHERE * ORDERBY DistanceFunc(column, reference) LIMIT count *Query without LIMIT count is not supported* - target_vector(should have float coordinates) examples: + reference(should have float coordinates) examples: tuple(0.1, 0.1, ...., 0.1) or (0.1, 0.1, ...., 0.1) [the word tuple is not needed] @@ -72,11 +72,11 @@ struct ApproximateNearestNeighborInformation returns true. From matching query it extracts - * targetVector + * referenceVector * metricName(DistanceFunction) * dimension size if query uses LpDistance * distance to compare(ONLY for search types, otherwise you get exception) - * spaceDimension(which is targetVector's components count) + * spaceDimension(which is reference vector's components count) * column * objects count from LIMIT clause(for both queries) * queryHasOrderByClause and queryHasWhereClause return true if query matches the type @@ -96,10 +96,10 @@ public: // returns the distance to compare with for search query float getComparisonDistanceForWhereQuery() const; - // distance should be calculated regarding to targetVector - std::vector getTargetVector() const; + // distance should be calculated regarding to reference vector + std::vector getReferenceVector() const; - // targetVector dimension size + // reference vector's dimension size size_t getNumOfDimensions() const; String getColumnName() const; @@ -196,7 +196,7 @@ private: // Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); - /* Matches dist function, target vector, column name */ + /* Matches dist function, reference vector, column name */ static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info); // Gets float or int from AST node diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 190f76fba5e..9dcfd421ba5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -247,7 +247,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI if (comp_dist && comp_dist.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); - std::vector target_vec = condition.getTargetVector(); + std::vector reference_vector = condition.getReferenceVector(); auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) @@ -260,13 +260,13 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI "does not match with the dimension in the index ({})", toString(condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); - /// neighbors contain indexes of dots which were closest to target vector + /// neighbors contain indexes of dots which were closest to the reference vector std::vector neighbors; std::vector distances; neighbors.reserve(limit); distances.reserve(limit); - annoy->get_nns_by_vector(target_vec.data(), limit, static_cast(search_k), &neighbors, &distances); + annoy->get_nns_by_vector(reference_vector.data(), limit, static_cast(search_k), &neighbors, &distances); std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) From 8cc382121a3b7580136eaf7514957dd0e08f3283 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:48:36 +0000 Subject: [PATCH 060/159] Cosmetics: Update comments --- src/Storages/MergeTree/CommonANNIndexes.cpp | 57 ++++---- src/Storages/MergeTree/CommonANNIndexes.h | 152 ++++++++++---------- 2 files changed, 103 insertions(+), 106 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 20707a148ae..f4a0e9bf728 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -126,7 +126,7 @@ ApproximateNearestNeighborInformation::Type ApproximateNearestNeighborCondition: bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryInfo & query) { - // RPN-s for different sections of the query + /// RPN-s for different sections of the query RPN rpn_prewhere_clause; RPN rpn_where_clause; RPN rpn_order_by_clause; @@ -137,47 +137,50 @@ bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryI ApproximateNearestNeighborInformation where_info; ApproximateNearestNeighborInformation order_by_info; - // Build rpns for query sections + /// Build rpns for query sections const auto & select = query.query->as(); - if (select.prewhere()) // If query has PREWHERE clause + /// If query has PREWHERE clause + if (select.prewhere()) traverseAST(select.prewhere(), rpn_prewhere_clause); - if (select.where()) // If query has WHERE clause + /// If query has WHERE clause + if (select.where()) traverseAST(select.where(), rpn_where_clause); - if (select.limitLength()) // If query has LIMIT clause + /// If query has LIMIT clause + if (select.limitLength()) traverseAtomAST(select.limitLength(), rpn_limit); if (select.orderBy()) // If query has ORDERBY clause traverseOrderByAST(select.orderBy(), rpn_order_by_clause); - // Reverse RPNs for conveniences during parsing + /// Reverse RPNs for conveniences during parsing std::reverse(rpn_prewhere_clause.begin(), rpn_prewhere_clause.end()); std::reverse(rpn_where_clause.begin(), rpn_where_clause.end()); std::reverse(rpn_order_by_clause.begin(), rpn_order_by_clause.end()); - // Match rpns with supported types and extract information + /// Match rpns with supported types and extract information const bool prewhere_is_valid = matchRPNWhere(rpn_prewhere_clause, prewhere_info); const bool where_is_valid = matchRPNWhere(rpn_where_clause, where_info); const bool order_by_is_valid = matchRPNOrderBy(rpn_order_by_clause, order_by_info); const bool limit_is_valid = matchRPNLimit(rpn_limit, limit); - // Query without a LIMIT clause or with a limit greater than a restriction is not supported + /// Query without a LIMIT clause or with a limit greater than a restriction is not supported if (!limit_is_valid || limit_restriction < limit) return false; - // Search type query in both sections isn't supported + /// Search type query in both sections isn't supported if (prewhere_is_valid && where_is_valid) return false; - // Search type should be in WHERE or PREWHERE clause + /// Search type should be in WHERE or PREWHERE clause if (prewhere_is_valid || where_is_valid) query_information = std::move(prewhere_is_valid ? prewhere_info : where_info); if (order_by_is_valid) { - // Query with valid where and order by type is not supported + /// Query with valid where and order by type is not supported if (query_information.has_value()) return false; @@ -202,7 +205,7 @@ void ApproximateNearestNeighborCondition::traverseAST(const ASTPtr & node, RPN & } RPNElement element; - // Get the data behind node + /// Get the data behind node if (!traverseAtomAST(node, element)) element.function = RPNElement::FUNCTION_UNKNOWN; @@ -211,10 +214,10 @@ void ApproximateNearestNeighborCondition::traverseAST(const ASTPtr & node, RPN & bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, RPNElement & out) { - // Match Functions + /// Match Functions if (const auto * function = node->as()) { - // Set the name + /// Set the name out.func_name = function->name; if (function->name == "L1Distance" || @@ -240,7 +243,7 @@ bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, R return true; } - // Match identifier + /// Match identifier else if (const auto * identifier = node->as()) { out.function = RPNElement::FUNCTION_IDENTIFIER; @@ -250,7 +253,7 @@ bool ApproximateNearestNeighborCondition::traverseAtomAST(const ASTPtr & node, R return true; } - // Check if we have constants behind the node + /// Check if we have constants behind the node return tryCastToConstType(node, out); } @@ -320,20 +323,20 @@ void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node traverseAST(order_by_element->children.front(), rpn); } -// Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause +/// Returns true and stores ApproximateNearestNeighborInformation if the query has valid WHERE clause bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; - // WHERE section must have at least 5 expressions - // Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) + /// WHERE section must have at least 5 expressions + /// Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) if (rpn.size() < 5) return false; auto iter = rpn.begin(); - // Query starts from operator less + /// Query starts from operator less if (iter->function != RPNElement::FUNCTION_COMPARISON) return false; @@ -371,11 +374,11 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe ann_info.reference_vector.pop_back(); } - // query is ok + /// query is ok return true; } -// Returns true and stores ANNExpr if the query has valid ORDERBY clause +/// Returns true and stores ANNExpr if the query has valid ORDERBY clause bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field @@ -391,7 +394,7 @@ bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, Approximate return ApproximateNearestNeighborCondition::matchMainParts(iter, end, ann_info); } -// Returns true and stores Length if we have valid LIMIT clause in query +/// Returns true and stores Length if we have valid LIMIT clause in query bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 & limit) { if (rpn.function == RPNElement::FUNCTION_INT_LITERAL) @@ -403,12 +406,12 @@ bool ApproximateNearestNeighborCondition::matchRPNLimit(RPNElement & rpn, UInt64 return false; } -/* Matches dist function, referencer vector, column name */ +/// Matches dist function, referencer vector, column name bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info) { bool identifier_found = false; - // Matches DistanceFunc->[Column]->[Tuple(array)Func]->ReferenceVector(floats)->[Column] + /// Matches DistanceFunc->[Column]->[Tuple(array)Func]->ReferenceVector(floats)->[Column] if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; @@ -487,11 +490,11 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; } - // Final checks of correctness + /// Final checks of correctness return identifier_found && !ann_info.reference_vector.empty(); } -// Gets float or int from AST node +/// Gets float or int from AST node float ApproximateNearestNeighborCondition::getFloatOrIntLiteralOrPanic(const RPN::iterator& iter) { if (iter->float_literal.has_value()) diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 9c075139707..2ac9384a884 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -9,31 +9,31 @@ namespace DB { -/** - * Queries for Approximate Nearest Neighbour Search - * have similar structure: - * 1) reference vector from which all distances are calculated - * 2) metric name (e.g L2Distance, LpDistance, etc.) - * 3) name of column with embeddings - * 4) type of query - * 5) Number of elements, that should be taken (limit) - * - * And two optional parameters: - * 1) p for LpDistance function - * 2) distance to compare with (only for where queries) - */ +/// Approximate Nearest Neighbour queries have a similar structure: +/// - reference vector from which all distances are calculated +/// - metric name (e.g L2Distance, LpDistance, etc.) +/// - name of column with embeddings +/// - type of query +/// - maximum number of returned elements (LIMIT) +/// +/// And two optional parameters: +/// - p for LpDistance function +/// - distance to compare with (only for where queries) +/// +/// This struct holds all these components. struct ApproximateNearestNeighborInformation { using Embedding = std::vector; - - // Extracted data from valid query Embedding reference_vector; + enum class Metric { Unknown, L2, Lp - } metric; + }; + Metric metric; + String column_name; UInt64 limit; @@ -41,79 +41,71 @@ struct ApproximateNearestNeighborInformation { OrderBy, Where - } query_type; + }; + Type query_type; float p_for_lp_dist = -1.0; float distance = -1.0; }; -/** - Class ApproximateNearestNeighborCondition, is responsible for recognizing special query types which - can be speeded up by ANN Indexes. It parses the SQL query and checks - if it matches ANNIndexes. The recognizing method - alwaysUnknownOrTrue - returns false if we can speed up the query, and true otherwise. - It has only one argument, name of the metric with which index was built. - There are two main patterns of queries being supported - - 1) Search query type - SELECT * FROM * WHERE DistanceFunc(column, reference) < floatLiteral LIMIT count - - 2) OrderBy query type - SELECT * FROM * WHERE * ORDERBY DistanceFunc(column, reference) LIMIT count - - *Query without LIMIT count is not supported* - - reference(should have float coordinates) examples: - tuple(0.1, 0.1, ...., 0.1) or (0.1, 0.1, ...., 0.1) - [the word tuple is not needed] - - If the query matches one of these two types, than the class extracts useful information - from the query. If the query has both 1 and 2 types, than we can't speed and alwaysUnknownOrTrue - returns true. - - From matching query it extracts - * referenceVector - * metricName(DistanceFunction) - * dimension size if query uses LpDistance - * distance to compare(ONLY for search types, otherwise you get exception) - * spaceDimension(which is reference vector's components count) - * column - * objects count from LIMIT clause(for both queries) - * queryHasOrderByClause and queryHasWhereClause return true if query matches the type - - Search query type is also recognized for PREWHERE clause -*/ +// Class ANNCondition, is responsible for recognizing if the query is an ANN queries which can utilize ANN indexes. It parses the SQL query +/// and checks if it matches ANNIndexes. Method alwaysUnknownOrTrue returns false if we can speed up the query, and true otherwise. It has +/// only one argument, the name of the metric with which index was built. Two main patterns of queries are supported +/// +/// - 1. WHERE queries: +/// SELECT * FROM * WHERE DistanceFunc(column, reference_vector) < floatLiteral LIMIT count +/// +/// - 2. ORDER BY queries: +/// SELECT * FROM * WHERE * ORDER BY DistanceFunc(column, reference_vector) LIMIT count +/// +/// Queries without LIMIT count are not supported +/// If the query is both of type 1. and 2., than we can't use the index and alwaysUnknownOrTrue returns true. +/// reference_vector should have float coordinates, e.g. (0.2, 0.1, .., 0.5) +/// +/// If the query matches one of these two types, then this class extracts the main information needed for ANN indexes from the query. +/// +/// From matching query it extracts +/// - referenceVector +/// - metricName(DistanceFunction) +/// - dimension size if query uses LpDistance +/// - distance to compare(ONLY for search types, otherwise you get exception) +/// - spaceDimension(which is referenceVector's components count) +/// - column +/// - objects count from LIMIT clause(for both queries) +/// - queryHasOrderByClause and queryHasWhereClause return true if query matches the type +/// +/// Search query type is also recognized for PREWHERE clause class ApproximateNearestNeighborCondition { public: ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context); - // false if query can be speeded up, true otherwise + /// Returns false if query can be speeded up by an ANN index, true otherwise. bool alwaysUnknownOrTrue(String metric_name) const; - // returns the distance to compare with for search query + /// Returns the distance to compare with for search query float getComparisonDistanceForWhereQuery() const; - // distance should be calculated regarding to reference vector + /// Distance should be calculated regarding to referenceVector std::vector getReferenceVector() const; - // reference vector's dimension size + /// Reference vector's dimension size size_t getNumOfDimensions() const; String getColumnName() const; ApproximateNearestNeighborInformation::Metric getMetricType() const; - // the P- value if the metric is 'LpDistance' + /// The P- value if the metric is 'LpDistance' float getPValueForLpDistance() const; ApproximateNearestNeighborInformation::Type getQueryType() const; UInt64 getIndexGranularity() const { return index_granularity; } - // length's value from LIMIT clause + /// Length's value from LIMIT clause UInt64 getLimit() const; private: @@ -121,7 +113,7 @@ private: { enum Function { - // DistanceFunctions + /// DistanceFunctions FUNCTION_DISTANCE, //tuple(0.1, ..., 0.1) @@ -130,31 +122,31 @@ private: //array(0.1, ..., 0.1) FUNCTION_ARRAY, - // Operators <, >, <=, >= + /// Operators <, >, <=, >= FUNCTION_COMPARISON, - // Numeric float value + /// Numeric float value FUNCTION_FLOAT_LITERAL, - // Numeric int value + /// Numeric int value FUNCTION_INT_LITERAL, - // Column identifier + /// Column identifier FUNCTION_IDENTIFIER, - // Unknown, can be any value + /// Unknown, can be any value FUNCTION_UNKNOWN, - // (0.1, ...., 0.1) vector without word 'tuple' + /// (0.1, ...., 0.1) vector without word 'tuple' FUNCTION_LITERAL_TUPLE, - // [0.1, ...., 0.1] vector without word 'array' + /// [0.1, ...., 0.1] vector without word 'array' FUNCTION_LITERAL_ARRAY, - // if client parameters are used, cast will always be in the query + /// if client parameters are used, cast will always be in the query FUNCTION_CAST, - // name of type in cast function + /// name of type in cast function FUNCTION_STRING_LITERAL, }; @@ -178,33 +170,33 @@ private: bool checkQueryStructure(const SelectQueryInfo & query); - // Util functions for the traversal of AST, parses AST and builds rpn + /// Util functions for the traversal of AST, parses AST and builds rpn void traverseAST(const ASTPtr & node, RPN & rpn); - // Return true if we can identify our node type + /// Return true if we can identify our node type bool traverseAtomAST(const ASTPtr & node, RPNElement & out); - // Checks if the AST stores ConstType expression + /// Checks if the AST stores ConstType expression bool tryCastToConstType(const ASTPtr & node, RPNElement & out); - // Traverses the AST of ORDERBY section + /// Traverses the AST of ORDERBY section void traverseOrderByAST(const ASTPtr & node, RPN & rpn); - // Returns true and stores ANNExpr if the query has valid WHERE section + /// Returns true and stores ANNExpr if the query has valid WHERE section static bool matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); - // Returns true and stores ANNExpr if the query has valid ORDERBY section + /// Returns true and stores ANNExpr if the query has valid ORDERBY section static bool matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info); - // Returns true and stores Length if we have valid LIMIT clause in query + /// Returns true and stores Length if we have valid LIMIT clause in query static bool matchRPNLimit(RPNElement & rpn, UInt64 & limit); /* Matches dist function, reference vector, column name */ static bool matchMainParts(RPN::iterator & iter, const RPN::iterator & end, ApproximateNearestNeighborInformation & ann_info); - // Gets float or int from AST node + /// Gets float or int from AST node static float getFloatOrIntLiteralOrPanic(const RPN::iterator& iter); Block block_with_constants; - // true if we have one of two supported query types + /// true if we have one of two supported query types std::optional query_information; // Get from settings ANNIndex parameters @@ -214,10 +206,12 @@ private: bool index_is_useful = false; }; -// condition interface for Ann indexes. Returns vector of indexes of ranges in granule which are useful for query. + +/// Common interface of ANN indexes. class IMergeTreeIndexConditionApproximateNearestNeighbor : public IMergeTreeIndexCondition { public: + /// Returns vector of indexes of ranges in granule which are useful for query. virtual std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const = 0; }; From 6580d2c326021cf476a274bce79b0b5d82bb92e4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:52:14 +0000 Subject: [PATCH 061/159] Cosmetics: castMetricFromStringToType --> stringToMetric --- src/Storages/MergeTree/CommonANNIndexes.cpp | 24 +-- src/Storages/MergeTree/CommonANNIndexes.h | 2 +- .../queries/0_stateless/02354_annoy.reference | 26 ---- tests/queries/0_stateless/02354_annoy.sh | 143 ------------------ 4 files changed, 14 insertions(+), 181 deletions(-) delete mode 100644 tests/queries/0_stateless/02354_annoy.reference delete mode 100755 tests/queries/0_stateless/02354_annoy.sh diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index f4a0e9bf728..2e2eb4e19ea 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -41,30 +41,32 @@ void extraceReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Em } } -ApproximateNearestNeighborInformation::Metric castMetricFromStringToType(String metric_name) +ApproximateNearestNeighborInformation::Metric stringToMetric(std::string_view metric) { - if (metric_name == "L2Distance") + if (metric == "L2Distance") return ApproximateNearestNeighborInformation::Metric::L2; - if (metric_name == "LpDistance") + else if (metric == "LpDistance") return ApproximateNearestNeighborInformation::Metric::Lp; - return ApproximateNearestNeighborInformation::Metric::Unknown; + else + return ApproximateNearestNeighborInformation::Metric::Unknown; } } ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) : - block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, - index_granularity{context->getMergeTreeSettings().get("index_granularity").get()}, - limit_restriction{context->getSettings().get("max_limit_for_ann_queries").get()}, - index_is_useful{checkQueryStructure(query_info)} {} + block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)), + index_granularity(context->getMergeTreeSettings().get("index_granularity").get()), + limit_restriction(context->getSettings().get("max_limit_for_ann_queries").get()), + index_is_useful(checkQueryStructure(query_info)) +{} -bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric_name) const +bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric) const { if (!index_is_useful) return true; // Query isn't supported // If query is supported, check metrics for match - return !(castMetricFromStringToType(metric_name) == query_information->metric); + return !(stringToMetric(metric) == query_information->metric); } float ApproximateNearestNeighborCondition::getComparisonDistanceForWhereQuery() const @@ -415,7 +417,7 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function != RPNElement::FUNCTION_DISTANCE) return false; - ann_info.metric = castMetricFromStringToType(iter->func_name); + ann_info.metric = stringToMetric(iter->func_name); ++iter; if (ann_info.metric == ApproximateNearestNeighborInformation::Metric::Lp) diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/CommonANNIndexes.h index 2ac9384a884..6b094a40b26 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.h +++ b/src/Storages/MergeTree/CommonANNIndexes.h @@ -83,7 +83,7 @@ public: ContextPtr context); /// Returns false if query can be speeded up by an ANN index, true otherwise. - bool alwaysUnknownOrTrue(String metric_name) const; + bool alwaysUnknownOrTrue(String metric) const; /// Returns the distance to compare with for search query float getComparisonDistanceForWhereQuery() const; diff --git a/tests/queries/0_stateless/02354_annoy.reference b/tests/queries/0_stateless/02354_annoy.reference deleted file mode 100644 index 38678fb67c9..00000000000 --- a/tests/queries/0_stateless/02354_annoy.reference +++ /dev/null @@ -1,26 +0,0 @@ -1 [0,0,10] -2 [0,0,10.5] -3 [0,0,9.5] -4 [0,0,9.7] -5 [0,0,10.2] -1 [0,0,10] -5 [0,0,10.2] -4 [0,0,9.7] -1 [0,0,10] -2 [0,0,10.5] -3 [0,0,9.5] -4 [0,0,9.7] -5 [0,0,10.2] -1 [0,0,10] -5 [0,0,10.2] -4 [0,0,9.7] - Name: annoy_index - Name: annoy_index -1 [0,0,10] -2 [0.2,0,10] -3 [-0.3,0,10] -1 [0,0,10] -2 [0.2,0,10] -3 [-0.3,0,10] - Name: annoy_index - Name: annoy_index diff --git a/tests/queries/0_stateless/02354_annoy.sh b/tests/queries/0_stateless/02354_annoy.sh deleted file mode 100755 index 1031ea81946..00000000000 --- a/tests/queries/0_stateless/02354_annoy.sh +++ /dev/null @@ -1,143 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# Check that index works correctly for L2Distance and with client parameters -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_l2; - -CREATE TABLE 02354_annoy_l2 -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy() GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -SELECT * -FROM 02354_annoy_l2 -WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 5; - -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; - -SET param_02354_target_vector='[0.0, 0.0, 10.0]'; - -SELECT * -FROM 02354_annoy_l2 -WHERE L2Distance(embedding, {02354_target_vector: Array(Float32)}) < 1.0 -LIMIT 5; - -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, {02354_target_vector: Array(Float32)}) -LIMIT 3; - -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, [0.0, 0.0]) -LIMIT 3; -- { serverError 80 } - - -DROP TABLE IF EXISTS 02354_annoy_l2; -" - -# Check that indexes are used -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_l2; - -CREATE TABLE 02354_annoy_l2 -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy() GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_l2 VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_l2 -WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 5; - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_l2 -ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; -DROP TABLE IF EXISTS 02354_annoy_l2; -" | grep "annoy_index" - - -# # Check that index works correctly for cosineDistance -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_cosine; - -CREATE TABLE 02354_annoy_cosine -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -SELECT * -FROM 02354_annoy_cosine -WHERE cosineDistance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 3; - -SELECT * -FROM 02354_annoy_cosine -ORDER BY cosineDistance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; - -DROP TABLE IF EXISTS 02354_annoy_cosine; -" - -# # Check that indexes are used -$CLICKHOUSE_CLIENT -nm --allow_experimental_annoy_index=1 -q " -DROP TABLE IF EXISTS 02354_annoy_cosine; - -CREATE TABLE 02354_annoy_cosine -( - id Int32, - embedding Array(Float32), - INDEX annoy_index embedding TYPE annoy('cosineDistance', 100) GRANULARITY 1 -) -ENGINE = MergeTree -ORDER BY id -SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; - -INSERT INTO 02354_annoy_cosine VALUES (1, [0.0, 0.0, 10.0]), (2, [0.2, 0.0, 10.0]), (3, [-0.3, 0.0, 10.0]), (4, [0.5, 0.0, 10.1]), (5, [0.8, 0.0, 10.0]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_cosine -WHERE cosineDistance(embedding, [0.0, 0.0, 10.0]) < 1.0 -LIMIT 3; - -EXPLAIN indexes=1 -SELECT * -FROM 02354_annoy_cosine -ORDER BY cosineDistance(embedding, [0.0, 0.0, 10.0]) -LIMIT 3; -DROP TABLE IF EXISTS 02354_annoy_cosine; -" | grep "annoy_index" From ee5b49c3fd71fddae0ca1d0e2ee96f3dadfb82f1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 21:57:17 +0000 Subject: [PATCH 062/159] Consolidate Annoy index tests --- .../0_stateless/02354_annoy_index.reference | 41 +++++++++++++++++++ .../queries/0_stateless/02354_annoy_index.sql | 34 +++++++++++++++ 2 files changed, 75 insertions(+) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 7da442cb905..5f3b523fbe4 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -1,3 +1,44 @@ +WHERE type, L2Distance +1 [0,0,10] +2 [0,0,10.5] +3 [0,0,9.5] +4 [0,0,9.7] +5 [0,0,10.2] +ORDER BY type, L2Distance +1 [0,0,10] +5 [0,0,10.2] +4 [0,0,9.7] +Reference ARRAYs with non-matching dimension are rejected +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/1 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/1 parameter annoy_index_search_k_nodes parameter max_limit_for_ann_queries Expression (Projection) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 3590b7d316e..3e1c176f3f6 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -9,6 +9,40 @@ CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding -- SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) +LIMIT 3; + + +SELECT 'Reference ARRAYs with non-matching dimension are rejected'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0]) +LIMIT 3; -- { serverError INCORRECT_QUERY } + + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) +LIMIT 3; + SELECT 'parameter annoy_index_search_k_nodes'; SELECT * FROM tab From 55256f4664c6226174fc249a652336b8d14a5251 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 22:12:54 +0000 Subject: [PATCH 063/159] Cosmetics: Fix typo --- src/Storages/MergeTree/CommonANNIndexes.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/CommonANNIndexes.cpp index 2e2eb4e19ea..45d86e54ea2 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/CommonANNIndexes.cpp @@ -25,7 +25,7 @@ namespace { template -void extraceReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & reference_vector, Literal literal) +void extractReferenceVectorFromLiteral(ApproximateNearestNeighborInformation::Embedding & reference_vector, Literal literal) { Float64 float_element_of_reference_vector; Int64 int_element_of_reference_vector; @@ -441,13 +441,13 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } @@ -462,12 +462,12 @@ bool ApproximateNearestNeighborCondition::matchMainParts(RPN::iterator & iter, c ++iter; if (iter->function == RPNElement::FUNCTION_LITERAL_TUPLE) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->tuple_literal); ++iter; } else if (iter->function == RPNElement::FUNCTION_LITERAL_ARRAY) { - extraceReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); + extractReferenceVectorFromLiteral(ann_info.reference_vector, iter->array_literal); ++iter; } else From 879b70a594fdc42421f04ff1f757a518b67c6956 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 08:50:36 +0000 Subject: [PATCH 064/159] CommonANNIndexes.h/cpp --> ApproximateNearestNeighborIndexesCommon.h/cpp --- ...es.cpp => ApproximateNearestNeighborIndexesCommon.cpp} | 8 +++----- ...ndexes.h => ApproximateNearestNeighborIndexesCommon.h} | 0 src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- 4 files changed, 5 insertions(+), 7 deletions(-) rename src/Storages/MergeTree/{CommonANNIndexes.cpp => ApproximateNearestNeighborIndexesCommon.cpp} (99%) rename src/Storages/MergeTree/{CommonANNIndexes.h => ApproximateNearestNeighborIndexesCommon.h} (100%) diff --git a/src/Storages/MergeTree/CommonANNIndexes.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp similarity index 99% rename from src/Storages/MergeTree/CommonANNIndexes.cpp rename to src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 45d86e54ea2..4f0bcd7ff81 100644 --- a/src/Storages/MergeTree/CommonANNIndexes.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -1,17 +1,15 @@ -#include -#include +#include +#include #include #include #include #include #include #include - +#include #include -#include - namespace DB { diff --git a/src/Storages/MergeTree/CommonANNIndexes.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h similarity index 100% rename from src/Storages/MergeTree/CommonANNIndexes.h rename to src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 3d290ea12ac..fb11fabfac1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -45,7 +45,7 @@ #include -#include +#include namespace CurrentMetrics { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 95041ea31fb..0ff676cf11f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -2,7 +2,7 @@ #ifdef ENABLE_ANNOY -#include +#include #include #include From 8213e366fae75c7079f814944d163e2e5d9b9b54 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 08:57:35 +0000 Subject: [PATCH 065/159] Cosmetics: Remove absolute namespace qualification of Annoy library internals, pt. II --- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 0ff676cf11f..2c78ff536ae 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -11,9 +11,9 @@ namespace DB { template -class AnnoyIndexWithSerialization : public ::Annoy::AnnoyIndex +class AnnoyIndexWithSerialization : public Annoy::AnnoyIndex { - using Base = ::Annoy::AnnoyIndex; + using Base = Annoy::AnnoyIndex; public: explicit AnnoyIndexWithSerialization(uint64_t dim); From 0c26123fd5cf1b848a14eff86d0e5a97461d4c16 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 26 May 2023 09:39:06 +0000 Subject: [PATCH 066/159] Cosmetics: limit_restriction --> max_limit_for_ann_queries --- .../ApproximateNearestNeighborIndexesCommon.cpp | 13 ++++++------- .../ApproximateNearestNeighborIndexesCommon.h | 5 ++--- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 4f0bcd7ff81..252035f3335 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -51,12 +51,11 @@ ApproximateNearestNeighborInformation::Metric stringToMetric(std::string_view me } -ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, - ContextPtr context) : - block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)), - index_granularity(context->getMergeTreeSettings().get("index_granularity").get()), - limit_restriction(context->getSettings().get("max_limit_for_ann_queries").get()), - index_is_useful(checkQueryStructure(query_info)) +ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) + : block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)) + , index_granularity(context->getMergeTreeSettings().get("index_granularity").get()) + , max_limit_for_ann_queries(context->getSettings().get("max_limit_for_ann_queries").get()) + , index_is_useful(checkQueryStructure(query_info)) {} bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric) const @@ -167,7 +166,7 @@ bool ApproximateNearestNeighborCondition::checkQueryStructure(const SelectQueryI const bool limit_is_valid = matchRPNLimit(rpn_limit, limit); /// Query without a LIMIT clause or with a limit greater than a restriction is not supported - if (!limit_is_valid || limit_restriction < limit) + if (!limit_is_valid || max_limit_for_ann_queries < limit) return false; /// Search type query in both sections isn't supported diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h index 6b094a40b26..513a86c42d8 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h @@ -79,8 +79,7 @@ struct ApproximateNearestNeighborInformation class ApproximateNearestNeighborCondition { public: - ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, - ContextPtr context); + ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context); /// Returns false if query can be speeded up by an ANN index, true otherwise. bool alwaysUnknownOrTrue(String metric) const; @@ -202,7 +201,7 @@ private: // Get from settings ANNIndex parameters const UInt64 index_granularity; /// only queries with a lower limit can be considered to avoid memory overflow - const UInt64 limit_restriction; + const UInt64 max_limit_for_ann_queries; bool index_is_useful = false; }; From 146668a850234394d34c03c2d0246bbed7855549 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 May 2023 15:34:43 +0000 Subject: [PATCH 067/159] Cosmetics: query_type --> type --- .../ApproximateNearestNeighborIndexesCommon.cpp | 8 ++++---- .../ApproximateNearestNeighborIndexesCommon.h | 10 ++++++++-- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index 252035f3335..c47e53788a7 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -69,7 +69,7 @@ bool ApproximateNearestNeighborCondition::alwaysUnknownOrTrue(String metric) con float ApproximateNearestNeighborCondition::getComparisonDistanceForWhereQuery() const { if (index_is_useful && query_information.has_value() - && query_information->query_type == ApproximateNearestNeighborInformation::Type::Where) + && query_information->type == ApproximateNearestNeighborInformation::Type::Where) return query_information->distance; throw Exception(ErrorCodes::LOGICAL_ERROR, "Not supported method for this query type"); } @@ -119,7 +119,7 @@ float ApproximateNearestNeighborCondition::getPValueForLpDistance() const ApproximateNearestNeighborInformation::Type ApproximateNearestNeighborCondition::getQueryType() const { if (index_is_useful && query_information.has_value()) - return query_information->query_type; + return query_information->type; throw Exception(ErrorCodes::LOGICAL_ERROR, "Query type was requested for useless or uninitialized index."); } @@ -326,7 +326,7 @@ void ApproximateNearestNeighborCondition::traverseOrderByAST(const ASTPtr & node bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - ann_info.query_type = ApproximateNearestNeighborInformation::Type::Where; + ann_info.type = ApproximateNearestNeighborInformation::Type::Where; /// WHERE section must have at least 5 expressions /// Operator->Distance(float)->DistanceFunc->Column->Tuple(Array)Func(ReferenceVector(floats)) @@ -381,7 +381,7 @@ bool ApproximateNearestNeighborCondition::matchRPNWhere(RPN & rpn, ApproximateNe bool ApproximateNearestNeighborCondition::matchRPNOrderBy(RPN & rpn, ApproximateNearestNeighborInformation & ann_info) { /// Fill query type field - ann_info.query_type = ApproximateNearestNeighborInformation::Type::OrderBy; + ann_info.type = ApproximateNearestNeighborInformation::Type::OrderBy; // ORDER BY clause must have at least 3 expressions if (rpn.size() < 3) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h index 513a86c42d8..4fb95c3f492 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h @@ -42,7 +42,7 @@ struct ApproximateNearestNeighborInformation OrderBy, Where }; - Type query_type; + Type type; float p_for_lp_dist = -1.0; float distance = -1.0; @@ -150,7 +150,11 @@ private: }; explicit RPNElement(Function function_ = FUNCTION_UNKNOWN) - : function(function_), func_name("Unknown"), float_literal(std::nullopt), identifier(std::nullopt) {} + : function(function_) + , func_name("Unknown") + , float_literal(std::nullopt) + , identifier(std::nullopt) + {} Function function; String func_name; @@ -200,8 +204,10 @@ private: // Get from settings ANNIndex parameters const UInt64 index_granularity; + /// only queries with a lower limit can be considered to avoid memory overflow const UInt64 max_limit_for_ann_queries; + bool index_is_useful = false; }; From c94ec9f5ddb8d047c370749f3249374fb746ed06 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 29 May 2023 15:35:14 +0000 Subject: [PATCH 068/159] Cosmetics: Fix typo --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3e10f48a2fb..df4cf5ff087 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -741,7 +741,7 @@ class IColumn; M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions (hashid, etc)", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ - M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexs.", 0) \ + M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ From 4e9a5331781166160feaf71ca8944a4f1b733846 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 09:53:20 +0000 Subject: [PATCH 069/159] Cosmetics: condition --> ann_condition --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 20 +++++++++---------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 9dcfd421ba5..9afaba6e521 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -208,12 +208,11 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( const SelectQueryInfo & query, const String & distance_function_, ContextPtr context) - : condition(query, context) + : ann_condition(query, context) , distance_function(distance_function_) , search_k(context->getSettings().get("annoy_index_search_k_nodes").get()) {} - bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const { throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes"); @@ -221,7 +220,7 @@ bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr / bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const { - return condition.alwaysUnknownOrTrue(distance_function); + return ann_condition.alwaysUnknownOrTrue(distance_function); } std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const @@ -234,20 +233,19 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndex throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); } - template std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const { - UInt64 limit = condition.getLimit(); - UInt64 index_granularity = condition.getIndexGranularity(); - std::optional comp_dist = condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where - ? std::optional(condition.getComparisonDistanceForWhereQuery()) + UInt64 limit = ann_condition.getLimit(); + UInt64 index_granularity = ann_condition.getIndexGranularity(); + std::optional comp_dist = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where + ? std::optional(ann_condition.getComparisonDistanceForWhereQuery()) : std::nullopt; if (comp_dist && comp_dist.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); - std::vector reference_vector = condition.getReferenceVector(); + std::vector reference_vector = ann_condition.getReferenceVector(); auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) @@ -255,10 +253,10 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI auto annoy = granule->index; - if (condition.getNumOfDimensions() != annoy->getNumOfDimensions()) + if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " "does not match with the dimension in the index ({})", - toString(condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); + toString(ann_condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); /// neighbors contain indexes of dots which were closest to the reference vector std::vector neighbors; diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 2c78ff536ae..457a505d909 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -81,7 +81,7 @@ private: template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; - const ApproximateNearestNeighborCondition condition; + const ApproximateNearestNeighborCondition ann_condition; const String distance_function; const Int64 search_k; }; From 397715bfa501e89de741e2b8edfbafdd0cede707 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 09:55:37 +0000 Subject: [PATCH 070/159] Cosmetics: comp_dist --> comparison_distance --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 9afaba6e521..7808f07511b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -238,11 +238,11 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI { UInt64 limit = ann_condition.getLimit(); UInt64 index_granularity = ann_condition.getIndexGranularity(); - std::optional comp_dist = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where + std::optional comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where ? std::optional(ann_condition.getComparisonDistanceForWhereQuery()) : std::nullopt; - if (comp_dist && comp_dist.value() < 0) + if (comparison_distance && comparison_distance.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); std::vector reference_vector = ann_condition.getReferenceVector(); @@ -269,7 +269,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI std::unordered_set granule_numbers; for (size_t i = 0; i < neighbors.size(); ++i) { - if (comp_dist && distances[i] > comp_dist) + if (comparison_distance && distances[i] > comparison_distance) continue; granule_numbers.insert(neighbors[i] / index_granularity); } From 06329fb08b8418e347d5c64a37beeb51d5504376 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 08:35:59 +0000 Subject: [PATCH 071/159] Minor: Make unique by sort/erase (should be a bit faster) --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 31 +++++++++---------- src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 2 +- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 7808f07511b..d1715b2c4c1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -148,8 +148,8 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t if (index_sample_block.columns() > 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); - auto index_column_name = index_sample_block.getByPosition(0).name; - const auto & column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); + const String & index_column_name = index_sample_block.getByPosition(0).name; + ColumnPtr column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); if (const auto & column_array = typeid_cast(column_cut.get())) { @@ -204,7 +204,7 @@ void MergeTreeIndexAggregatorAnnoy::update(const Block & block, size_t MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( - const IndexDescription & /*index*/, + const IndexDescription & /*index_description*/, const SelectQueryInfo & query, const String & distance_function_, ContextPtr context) @@ -251,35 +251,34 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); - auto annoy = granule->index; + const AnnoyIndexWithSerializationPtr & annoy = granule->index; if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " - "does not match with the dimension in the index ({})", - toString(ann_condition.getNumOfDimensions()), toString(annoy->getNumOfDimensions())); + "does not match the dimension in the index ({})", + ann_condition.getNumOfDimensions(), annoy->getNumOfDimensions()); - /// neighbors contain indexes of dots which were closest to the reference vector - std::vector neighbors; + std::vector neighbors; /// indexes of dots which were closest to the reference vector std::vector distances; neighbors.reserve(limit); distances.reserve(limit); annoy->get_nns_by_vector(reference_vector.data(), limit, static_cast(search_k), &neighbors, &distances); - std::unordered_set granule_numbers; + std::vector granule_numbers; + granule_numbers.reserve(neighbors.size()); for (size_t i = 0; i < neighbors.size(); ++i) { if (comparison_distance && distances[i] > comparison_distance) continue; - granule_numbers.insert(neighbors[i] / index_granularity); + granule_numbers.push_back(neighbors[i] / index_granularity); } - std::vector result_vector; - result_vector.reserve(granule_numbers.size()); - for (auto granule_number : granule_numbers) - result_vector.push_back(granule_number); + /// make unique + std::sort(granule_numbers.begin(), granule_numbers.end()); + granule_numbers.erase(std::unique(granule_numbers.begin(), granule_numbers.end()), granule_numbers.end()); - return result_vector; + return granule_numbers; } MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, uint64_t trees_, const String & distance_function_) @@ -302,7 +301,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 if (distance_function == "L2Distance") return std::make_shared>(index.name, index.sample_block, trees); - if (distance_function == "cosineDistance") + else if (distance_function == "cosineDistance") return std::make_shared>(index.name, index.sample_block, trees); std::unreachable(); } diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index 457a505d909..bca06edd0f8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -64,7 +64,7 @@ class MergeTreeIndexConditionAnnoy final : public IMergeTreeIndexConditionApprox { public: MergeTreeIndexConditionAnnoy( - const IndexDescription & index, + const IndexDescription & index_description, const SelectQueryInfo & query, const String& distance_function, ContextPtr context); From 22110ac7427abc4feac120a1cfa5cd29850def8b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 5 Jun 2023 11:59:49 +0200 Subject: [PATCH 072/159] Fix exception message --- src/Interpreters/Cache/Metadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 9dff77e2af8..d0780202121 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -374,7 +374,7 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm fs::remove(path); } else if (file_segment->downloaded_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist", path); file_segment->detach(segment_lock, *this); return key_metadata->erase(it); From cdb5997339f20f27d6c6ad628c341579ffcf9264 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:22:34 +0000 Subject: [PATCH 073/159] Cosmetics: add assert --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 -- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 4 +++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index fb11fabfac1..c6b7232be4a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1693,9 +1693,7 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( // vector of indexes of useful ranges auto result = ann_condition->getUsefulRanges(granule); if (result.empty()) - { ++granules_dropped; - } for (auto range : result) { diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index d1715b2c4c1..cf7fbb3bab3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -251,7 +251,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); - const AnnoyIndexWithSerializationPtr & annoy = granule->index; + AnnoyIndexWithSerializationPtr annoy = granule->index; if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " @@ -265,6 +265,8 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI annoy->get_nns_by_vector(reference_vector.data(), limit, static_cast(search_k), &neighbors, &distances); + chassert(neighbors.size() == distances.size()); + std::vector granule_numbers; granule_numbers.reserve(neighbors.size()); for (size_t i = 0; i < neighbors.size(); ++i) From 7d659fdca2bf80c6f045333f531a8a4f058d33d8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:26:45 +0000 Subject: [PATCH 074/159] Tests: Decrease granule size --- tests/queries/0_stateless/02354_annoy_index.reference | 10 +++++----- tests/queries/0_stateless/02354_annoy_index.sql | 7 +------ 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 5f3b523fbe4..5b37c32d914 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -17,12 +17,12 @@ Expression ((Projection + Before ORDER BY)) PrimaryKey Condition: true Parts: 1/1 - Granules: 1/1 + Granules: 3/3 Skip Name: annoy_index Description: annoy GRANULARITY 1 Parts: 1/1 - Granules: 1/1 + Granules: 1/3 ORDER BY type, L2Distance, check that index is used Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) @@ -33,12 +33,12 @@ Expression (Projection) PrimaryKey Condition: true Parts: 1/1 - Granules: 1/1 + Granules: 3/3 Skip Name: annoy_index Description: annoy GRANULARITY 1 Parts: 1/1 - Granules: 1/1 + Granules: 3/3 parameter annoy_index_search_k_nodes parameter max_limit_for_ann_queries Expression (Projection) @@ -50,5 +50,5 @@ Expression (Projection) PrimaryKey Condition: true Parts: 1/1 - Granules: 1/1 + Granules: 3/3 Negative tests diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 3e1c176f3f6..2c40653667d 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -3,10 +3,7 @@ SET allow_experimental_annoy_index = 1; DROP TABLE IF EXISTS tab; - -DROP TABLE IF EXISTS tab; -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; --- SETTINGS index_granularity=5, index_granularity_bytes = '10Mi'; +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); SELECT 'WHERE type, L2Distance'; @@ -21,14 +18,12 @@ FROM tab ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) LIMIT 3; - SELECT 'Reference ARRAYs with non-matching dimension are rejected'; SELECT * FROM tab ORDER BY L2Distance(embedding, [0.0, 0.0]) LIMIT 3; -- { serverError INCORRECT_QUERY } - SELECT 'WHERE type, L2Distance, check that index is used'; EXPLAIN indexes=1 SELECT * From de503dc5d20bb8ddd8816c9639686c2c6089fdbe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:49:45 +0000 Subject: [PATCH 075/159] Add tests for tuple + add tests for custom distance metric / tree count --- .../0_stateless/02354_annoy_index.reference | 67 ++++++++++++++- .../queries/0_stateless/02354_annoy_index.sql | 81 +++++++++++++++++-- 2 files changed, 140 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 5b37c32d914..5bd1377d6f4 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -1,3 +1,4 @@ +--- Test with Array --- WHERE type, L2Distance 1 [0,0,10] 2 [0,0,10.5] @@ -51,4 +52,68 @@ Expression (Projection) Condition: true Parts: 1/1 Granules: 3/3 -Negative tests +--- Test with Tuple --- +WHERE type, L2Distance +1 (0,0,10) +2 (0,0,10.5) +3 (0,0,9.5) +4 (0,0,9.7) +5 (0,0,10.2) +ORDER BY type, L2Distance +1 (0,0,10) +5 (0,0,10.2) +4 (0,0,9.7) +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 1/3 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: annoy_index + Description: annoy GRANULARITY 1 + Parts: 1/1 + Granules: 3/3 +parameter annoy_index_search_k_nodes +parameter max_limit_for_ann_queries +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 +--- Test alternative metric (cosine distance) and non-default NumTrees --- +WHERE type, L2Distance +1 [0,0,10] +2 [0,0,10.5] +3 [0,0,9.5] +4 [0,0,9.7] +5 [0,0,10.2] +ORDER BY type, L2Distance +1 [0,0,10] +5 [0,0,10.2] +4 [0,0,9.7] +--- Negative tests --- diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 2c40653667d..170c048d420 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -2,6 +2,8 @@ SET allow_experimental_annoy_index = 1; +SELECT '--- Test with Array ---'; + DROP TABLE IF EXISTS tab; CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); @@ -55,19 +57,87 @@ SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index DROP TABLE tab; -DROP TABLE IF EXISTS tab; +SELECT '--- Test with Tuple ---'; -SELECT 'Negative tests'; +CREATE TABLE tab(id Int32, embedding Tuple(Float32, Float32, Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; +INSERT INTO tab VALUES (1, (0.0, 0.0, 10.0)), (2, (0.0, 0.0, 10.5)), (3, (0.0, 0.0, 9.5)), (4, (0.0, 0.0, 9.7)), (5, (0.0, 0.0, 10.2)), (6, (10.0, 0.0, 0.0)), (7, (9.5, 0.0, 0.0)), (8, (9.7, 0.0, 0.0)), (9, (10.2, 0.0, 0.0)), (10, (10.5, 0.0, 0.0)), (11, (0.0, 10.0, 0.0)), (12, (0.0, 9.5, 0.0)), (13, (0.0, 9.7, 0.0)), (14, (0.0, 10.2, 0.0)), (15, (0.0, 10.5, 0.0)); + +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0)) +LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(embedding, (0.0, 0.0, 10.0)) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (0.0, 0.0, 10.0)) +LIMIT 3; + +SELECT 'parameter annoy_index_search_k_nodes'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1)) +LIMIT 5 +SETTINGS annoy_index_search_k_nodes=0; -- searches zero nodes --> no results + +SELECT 'parameter max_limit_for_ann_queries'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(embedding, (5.3, 7.3, 2.1)) +LIMIT 5 +SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index + +DROP TABLE tab; + +SELECT '--- Test alternative metric (cosine distance) and non-default NumTrees ---'; + +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('cosineDistance', 200)) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity=5; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(embedding, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 5; + +SELECT 'ORDER BY type, L2Distance'; +SELECT * +FROM tab +ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) +LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Negative tests ---'; -- must have at most 2 arguments CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } --- first argument must be UInt64 +-- first argument (distance_function) must be String CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } --- 2nd argument must be String +-- 2nd argument (number of trees) must be UInt64 CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('L2Distance', 'not an UInt64')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +-- reject unsupported distance functions +CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } + -- must be created on single column CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index (embedding, id) TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } @@ -77,6 +147,3 @@ CREATE TABLE tab(id Int32, embedding Float32, INDEX annoy_index embedding TYPE a CREATE TABLE tab(id Int32, embedding Array(Float64), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding LowCardinality(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } CREATE TABLE tab(id Int32, embedding Nullable(Float32), INDEX annoy_index embedding TYPE annoy()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } - --- reject unsupported distance functions -CREATE TABLE tab(id Int32, embedding Array(Float32), INDEX annoy_index embedding TYPE annoy('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } From 8f60423daa828d9f4ce8f13b90e1c42de22f9edd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 10:55:05 +0000 Subject: [PATCH 076/159] Cosmetics: more constness --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 12 ++++++------ src/Storages/MergeTree/MergeTreeIndexAnnoy.h | 5 +---- 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index cf7fbb3bab3..72dd92ead5e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -236,22 +236,22 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndex template std::vector MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const { - UInt64 limit = ann_condition.getLimit(); - UInt64 index_granularity = ann_condition.getIndexGranularity(); - std::optional comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where + const UInt64 limit = ann_condition.getLimit(); + const UInt64 index_granularity = ann_condition.getIndexGranularity(); + const std::optional comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where ? std::optional(ann_condition.getComparisonDistanceForWhereQuery()) : std::nullopt; if (comparison_distance && comparison_distance.value() < 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); - std::vector reference_vector = ann_condition.getReferenceVector(); + const std::vector reference_vector = ann_condition.getReferenceVector(); - auto granule = std::dynamic_pointer_cast>(idx_granule); + const auto granule = std::dynamic_pointer_cast>(idx_granule); if (granule == nullptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); - AnnoyIndexWithSerializationPtr annoy = granule->index; + const AnnoyIndexWithSerializationPtr annoy = granule->index; if (ann_condition.getNumOfDimensions() != annoy->getNumOfDimensions()) throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h index bca06edd0f8..5204ff07b27 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.h +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.h @@ -66,15 +66,13 @@ public: MergeTreeIndexConditionAnnoy( const IndexDescription & index_description, const SelectQueryInfo & query, - const String& distance_function, + const String & distance_function, ContextPtr context); ~MergeTreeIndexConditionAnnoy() override = default; bool alwaysUnknownOrTrue() const override; - bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; - std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const override; private: @@ -97,7 +95,6 @@ public: MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator() const override; - MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } From d9a6e36685f71ed592af24a7e951272a688b9eea Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 5 Jun 2023 11:02:23 +0000 Subject: [PATCH 077/159] Add comments --- src/Analyzer/TableNode.cpp | 3 +++ src/Planner/PlannerJoinTree.cpp | 3 +++ src/Planner/Utils.cpp | 3 +++ src/Storages/StorageDistributed.cpp | 5 +++-- 4 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/TableNode.cpp b/src/Analyzer/TableNode.cpp index 17d12bd6afa..f899c1ae6fe 100644 --- a/src/Analyzer/TableNode.cpp +++ b/src/Analyzer/TableNode.cpp @@ -91,6 +91,9 @@ ASTPtr TableNode::toASTImpl(const ConvertToASTOptions & /* options */) const if (!temporary_table_name.empty()) return std::make_shared(temporary_table_name); + // In case of cross-replication we don't know what database is used for the table. + // `storage_id.hasDatabase()` can return false only on the initiator node. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). if (!storage_id.hasDatabase()) return std::make_shared(storage_id.getTableName()); return std::make_shared(storage_id.getDatabaseName(), storage_id.getTableName()); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4a1708f96d3..8bd674f5b67 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -106,6 +106,9 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, storage_id.getFullTableName()); } + // In case of cross-replication we don't know what database is used for the table. + // `storage_id.hasDatabase()` can return false only on the initiator node. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). if (storage_id.hasDatabase()) query_context->checkAccess(AccessType::SELECT, storage_id, column_names); } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index cd4fb9182e9..94f3bbf6440 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -107,6 +107,9 @@ Block buildCommonHeaderForUnion(const Blocks & queries_headers, SelectUnionMode ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) { auto & query_node_typed = query_node->as(); + + // In case of cross-replication we don't know what database is used for the table. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). auto result_ast = query_node_typed.toAST({ .fully_qualified_identifiers = false }); while (true) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b9625ce2ab7..969c117cb28 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,8 +30,6 @@ #include #include #include -#include "Analyzer/IQueryTreeNode.h" -#include "Analyzer/MatcherNode.h" #include #include @@ -946,6 +944,9 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, else { auto resolved_remote_storage_id = remote_storage_id; + // In case of cross-replication we don't know what database is used for the table. + // `storage_id.hasDatabase()` can return false only on the initiator node. + // Each shard will use the default database (in the case of cross-replication shards may have different defaults). if (remote_storage_id.hasDatabase()) resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); From c8d85a43c70f96de5c5ce922a43897425566aa00 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 5 Jun 2023 12:22:32 +0000 Subject: [PATCH 078/159] Fix unit tests --- src/Analyzer/ColumnNode.cpp | 4 ++-- src/Analyzer/IQueryTreeNode.h | 5 ++++- src/Planner/Utils.cpp | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/ColumnNode.cpp b/src/Analyzer/ColumnNode.cpp index dd41522ac7d..3d9f5d1640e 100644 --- a/src/Analyzer/ColumnNode.cpp +++ b/src/Analyzer/ColumnNode.cpp @@ -96,7 +96,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const std::vector column_identifier_parts; auto column_source = getColumnSourceOrNull(); - if (column_source) + if (column_source && options.fully_qualified_identifiers) { auto node_type = column_source->getNodeType(); if (node_type == QueryTreeNodeType::TABLE || @@ -117,7 +117,7 @@ ASTPtr ColumnNode::toASTImpl(const ConvertToASTOptions & options) const else { const auto & table_storage_id = table_node->getStorageID(); - if (table_storage_id.hasDatabase() && options.fully_qualified_identifiers) + if (table_storage_id.hasDatabase() && options.qualify_indentifiers_with_database) column_identifier_parts = { table_storage_id.getDatabaseName(), table_storage_id.getTableName() }; else column_identifier_parts = { table_storage_id.getTableName() }; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 351d03bc8cb..763963b734a 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -187,10 +187,13 @@ public: /// Identifiers are fully qualified (`database.table.column`), otherwise names are just column names (`column`) bool fully_qualified_identifiers = true; + + /// Identifiers are qualified but database name is not added (`table.column`) if set to false. + bool qualify_indentifiers_with_database = true; }; /// Convert query tree to AST - ASTPtr toAST(const ConvertToASTOptions & options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true }) const; + ASTPtr toAST(const ConvertToASTOptions & options = { .add_cast_for_constants = true, .fully_qualified_identifiers = true, .qualify_indentifiers_with_database = true }) const; /// Convert query tree to AST and then format it for error message. String formatConvertedASTForErrorMessage() const; diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 94f3bbf6440..733db0f00bc 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -110,7 +110,7 @@ ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) // In case of cross-replication we don't know what database is used for the table. // Each shard will use the default database (in the case of cross-replication shards may have different defaults). - auto result_ast = query_node_typed.toAST({ .fully_qualified_identifiers = false }); + auto result_ast = query_node_typed.toAST({ .qualify_indentifiers_with_database = false }); while (true) { From 4f0adf5f61fca0162184e5d4858c75fdb0a10e2e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 12:40:54 +0000 Subject: [PATCH 079/159] Better support for avro decimals --- .../Formats/Impl/AvroRowInputFormat.cpp | 5 +++-- .../0_stateless/02782_avro_decimals.reference | 13 +++++++++++++ tests/queries/0_stateless/02782_avro_decimals.sh | 10 ++++++++++ tests/queries/0_stateless/data_avro/decimals.avro | Bin 0 -> 295 bytes 4 files changed, 26 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02782_avro_decimals.reference create mode 100755 tests/queries/0_stateless/02782_avro_decimals.sh create mode 100644 tests/queries/0_stateless/data_avro/decimals.avro diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index c2602a4d1d5..267f9e522e2 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -176,14 +176,15 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No { static constexpr size_t field_type_size = sizeof(typename DecimalType::FieldType); decoder.decodeString(tmp); - if (tmp.size() != field_type_size) + if (tmp.size() > field_type_size) throw ParsingException( ErrorCodes::CANNOT_PARSE_UUID, - "Cannot parse type {}, expected binary data with size {}, got {}", + "Cannot parse type {}, expected binary data with size less then {}, got {}", target_type->getName(), field_type_size, tmp.size()); + tmp = std::string(field_type_size - tmp.size(), '\0') + tmp; typename DecimalType::FieldType field; ReadBufferFromString buf(tmp); readBinaryBigEndian(field.value, buf); diff --git a/tests/queries/0_stateless/02782_avro_decimals.reference b/tests/queries/0_stateless/02782_avro_decimals.reference new file mode 100644 index 00000000000..ed46f1c3758 --- /dev/null +++ b/tests/queries/0_stateless/02782_avro_decimals.reference @@ -0,0 +1,13 @@ +d Decimal(14, 4) +0 +1 +1.1 +12.12 +123.123 +1234.1234 +12345.1234 +123456.1234 +1234567.1234 +12345678.1234 +123456789.1234 +1234567890.1234 diff --git a/tests/queries/0_stateless/02782_avro_decimals.sh b/tests/queries/0_stateless/02782_avro_decimals.sh new file mode 100755 index 00000000000..5b754965806 --- /dev/null +++ b/tests/queries/0_stateless/02782_avro_decimals.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "desc file('$CUR_DIR/data_avro/decimals.avro')" +$CLICKHOUSE_LOCAL -q "select * from file('$CUR_DIR/data_avro/decimals.avro')" + diff --git a/tests/queries/0_stateless/data_avro/decimals.avro b/tests/queries/0_stateless/data_avro/decimals.avro new file mode 100644 index 0000000000000000000000000000000000000000..5c29ac235d59003696735c8c8092eed5bcce41b2 GIT binary patch literal 295 zcmeZI%3@>@Nh~YM*GtY%NloU+E6vFf1M`cMGg5OCFECXrB@y Date: Mon, 5 Jun 2023 12:43:38 +0000 Subject: [PATCH 080/159] Fix exception message --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 267f9e522e2..ae65960a372 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -179,7 +179,7 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No if (tmp.size() > field_type_size) throw ParsingException( ErrorCodes::CANNOT_PARSE_UUID, - "Cannot parse type {}, expected binary data with size less then {}, got {}", + "Cannot parse type {}, expected binary data with size equal to or less than {}, got {}", target_type->getName(), field_type_size, tmp.size()); From aa20935cb9913d628df08e80b793ad853363203a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 5 Jun 2023 12:45:14 +0000 Subject: [PATCH 081/159] Better --- src/Processors/Formats/Impl/AvroRowInputFormat.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index ae65960a372..201845177a5 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -183,8 +183,10 @@ static AvroDeserializer::DeserializeFn createDecimalDeserializeFn(const avro::No target_type->getName(), field_type_size, tmp.size()); + else if (tmp.size() != field_type_size) + /// Add padding with 0-bytes. + tmp = std::string(field_type_size - tmp.size(), '\0') + tmp; - tmp = std::string(field_type_size - tmp.size(), '\0') + tmp; typename DecimalType::FieldType field; ReadBufferFromString buf(tmp); readBinaryBigEndian(field.value, buf); From 638775e5802df2393c020d9409855ab58d7ff54b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 5 Jun 2023 12:48:54 +0000 Subject: [PATCH 082/159] Analyzer: fix 01487_distributed_in_not_default_db --- tests/broken_tests.txt | 1 - .../0_stateless/01487_distributed_in_not_default_db.sql | 4 ++++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index 02935712325..da2493faa1e 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -37,7 +37,6 @@ 01319_optimize_skip_unused_shards_nesting 01353_low_cardinality_join_types 01455_shard_leaf_max_rows_bytes_to_read -01487_distributed_in_not_default_db 01495_subqueries_in_with_statement 01504_rocksdb 01527_dist_sharding_key_dictGet_reload diff --git a/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql b/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql index ccd2c571290..cd027530ac8 100644 --- a/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql +++ b/tests/queries/0_stateless/01487_distributed_in_not_default_db.sql @@ -25,6 +25,10 @@ CREATE TABLE d AS t ENGINE = Distributed(test_cluster_two_shards_different_datab USE test_01487; DROP DATABASE test_01487; +-- After the default database is dropped QueryAnalysisPass cannot process the following SELECT query. +-- That query is invalid on the initiator node. +set allow_experimental_analyzer = 0; + SELECT * FROM main_01487.d WHERE value IN (SELECT l.value FROM l) ORDER BY value; USE main_01487; From c4e1dc55458cfc53ad3eee1406e6bac61bfe32d7 Mon Sep 17 00:00:00 2001 From: Daniel Kutenin Date: Mon, 5 Jun 2023 14:10:53 +0100 Subject: [PATCH 083/159] Enable FAST_DEC_LOOP for Arm LZ4 to get 5% of decomp speed It's disabled for clang because of mobile https://github.com/lz4/lz4/blob/e82198428c8061372d5adef1f9bfff4203f6081e/lib/lz4.c#L471 --- contrib/lz4-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/lz4-cmake/CMakeLists.txt b/contrib/lz4-cmake/CMakeLists.txt index 94def029410..c0fd574134f 100644 --- a/contrib/lz4-cmake/CMakeLists.txt +++ b/contrib/lz4-cmake/CMakeLists.txt @@ -12,6 +12,7 @@ add_library (_lz4 ${SRCS}) add_library (ch_contrib::lz4 ALIAS _lz4) target_compile_definitions (_lz4 PUBLIC LZ4_DISABLE_DEPRECATE_WARNINGS=1) +target_compile_definitions (_lz4 PUBLIC LZ4_FAST_DEC_LOOP=1) if (SANITIZE STREQUAL "undefined") target_compile_options (_lz4 PRIVATE -fno-sanitize=undefined) endif () From 4c88b7bbb725245060067b101593688f470ee399 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 5 Jun 2023 13:13:49 +0000 Subject: [PATCH 084/159] Further improve ANN index docs --- .../mergetree-family/annindexes.md | 152 +++++++++--------- 1 file changed, 72 insertions(+), 80 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 0cc1cff2dad..58655c11321 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -22,15 +22,53 @@ ORDER BY L2Distance(column, Point) LIMIT N ``` -The queries are expensive because the L2 distance (Euclidean distance) between all points in `column` and `Point` must be computed. To speed this process up, ANN indexes store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly. +The queries are expensive because the L2 (Euclidean) distance between `Point` and all points in `column` and must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer quickly. -## Indexes Structure +# Creating ANN Indexes -Approximate Nearest Neighbor Search Indexes (or `ANNIndexes`) are similar to skip indexes. They are constructed over granules and determine which granules can be skipped. Compared to skip indices, ANN indices are not only able to skip granules, they can also to select particular granules from a set of granules. +As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`. -`ANNIndexes` support two types of queries: +Syntax to create an ANN index over an `Array` column: + +```sql +CREATE TABLE table +( + `id` Int64, + `embedding` Array(Float32), + INDEX embedding TYPE () GRANULARITY +) +ENGINE = MergeTree +ORDER BY id; +``` + +Syntax to create an ANN index over a `Tuple` column: + +```sql +CREATE TABLE table +( + `id` Int64, + `embedding` Tuple(Float32[, Float32[, ...]]), + INDEX embedding TYPE () GRANULARITY +) +ENGINE = MergeTree +ORDER BY id; +``` + +ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively comparatively many more read requests than write requests. + +Similar to regular skip indexes, ANN indexes are constructed over granules and each indexed block consists of `GRANULARITY = `-many +granules. For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, +then each indexed block will consist of 16384 rows. However, unlike skip indexes, ANN indexes are not only able to skip the entire indexed +block, they are able to skip individual granules in indexed blocks. As a result, the `GRANULARITY` parameter has a different meaning in ANN +indexes than in normal skip indexes. Basically, the bigger `GRANULARITY` is chosen, the more data is provided to a single ANN index, and the +higher the chance that with the right hyper parameters, the index will remember the data structure better. + +# Using ANN Indexes + +ANN indexes support two types of queries: - WHERE queries: + ``` sql SELECT * FROM table @@ -39,86 +77,40 @@ Approximate Nearest Neighbor Search Indexes (or `ANNIndexes`) are similar to ski ``` - ORDER BY queries: + ``` sql SELECT * - FROM table [WHERE ...] + FROM table + [WHERE ...] ORDER BY DistanceFunction(column, Point) LIMIT N ``` -`DistanceFunction` is a [distance functions](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a given vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a float value which restricts the size of the neighbourhood. +`DistanceFunction` is a [distance function](/docs/en/sql-reference/functions/distance-functions.md), `Point` is a reference vector (e.g. `(0.17, 0.33, ...)`) and `MaxDistance` is a floating point value which restricts the size of the neighbourhood. -To avoid writing large vectors, you can also use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. +:::tip +To avoid writing out large vectors, you can use [query parameters](/docs/en//interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. ```bash clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(embedding, {vec: Array(Float32)}) < 1.0" ``` +::: -ANN index cannot speed up query that contain both `WHERE` and `ORDER BY`. Queries must have a limit, as the approximate algorithms used to determine the nearest neighbors require a specific number of them. +ANN indexes cannot speed up queries that contain both a `WHERE DistanceFunction(column, Point) < MaxDistance` and an `ORDER BY DistanceFunction(column, Point)` clause. Also, the approximate algorithms used to determine the nearest neighbors require a limit, hence queries that use an ANN index must have a `LIMIT` clause. -Indexes are only used for queries with a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This helps to prevent memory overflows in queries with a large limit. +An ANN index is only used if the query has a `LIMIT` value smaller than setting `max_limit_for_ann_queries` (default: 1 million rows). This is a safety measure which helps to avoid large memory consumption by external libraries for approximate neighbor search. -Both types of queries are processed similarly. The indexes are passed the number of neighbors `N`. In `ORDER BY` query they remember the numbers of all parts of the granule that have at least one of neighbor. In `WHERE` query they remember only those parts that satisfy the requirements. - - -## Creating Tables with an ANN Index - -As long as ANN indexes are experimental, you first need to `SET allow_experimental_annoy_index = 1`. - -Syntax: - -```sql -CREATE TABLE table -( - `id` Int64, - `embedding` Tuple(Float32, Float32, Float32), - INDEX embedding TYPE () GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` - -```sql -CREATE TABLE table -( - `id` Int64, - `embedding` Array(Float32), - INDEX embedding TYPE () GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` - -With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. - -Note that ANN indexes are built during column insertion and merge, i.e. `INSERT` and `OPTIMIZE` statements are slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changing data in conjunction with many read requests. - -# Index list +# Available ANN Indexes - [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) -# Annoy {#annoy} +## Annoy {#annoy} -(currently disabled on ARM due to problems with the algorithm) +(currently disabled on ARM due to memory safety problems with the algorithm) -This ANN index type implements [Annoy indexes](https://github.com/spotify/annoy). +This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which uses a recursive division of the space in random linear surfaces (lines in 2D, planes in 3D etc.). -Short description of the algorithm: -The algorithm recursively divides in half all space by random linear surfaces (lines in 2D, planes in 3D etc.). Thus it makes tree of polyhedrons and points that they contains. Repeating the operation several times for greater accuracy it creates a forest. -To find K Nearest Neighbours it goes down through the trees and fills the buffer of closest points using the priority queue of polyhedrons. Next, it sorts buffer and return the nearest K points. - -Examples: - -```sql -CREATE TABLE table -( - id Int64, - embedding Tuple(Float32, Float32, Float32), - INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` +Syntax to create a Annoy index over a `Array` column: ```sql CREATE TABLE table @@ -131,26 +123,26 @@ ENGINE = MergeTree ORDER BY id; ``` +Syntax to create a Annoy index over a `Tuple` column: + +```sql +CREATE TABLE table +( + id Int64, + embedding Tuple(Float32[, Float32[, ...]]), + INDEX embedding TYPE annoy([DistanceName[, NumTrees]]) GRANULARITY N +) +ENGINE = MergeTree +ORDER BY id; +``` + +Parameter `DistanceName` is name of a distance function (default `L2Distance`). Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results. + :::note Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use [CONSTRAINT](/docs/en/sql-reference/statements/create/table.md#constraints) to avoid errors. For example, `CONSTRAINT constraint_name_1 CHECK length(embedding) = 256`. ::: -Parameter `DistanceName` is name of a distance function with default `L2Distance`. Parameter `NumTrees` (default: 100) is the number of trees which the algorithm will create. Higher values of `NumTree` mean slower `CREATE` and `SELECT` statements (approximately linearly), but increase the accuracy of search results. - -```sql -CREATE TABLE table -( - id Int64, - embedding Array(Float32), - INDEX ann_index_name embedding TYPE annoy('cosineDistance') GRANULARITY N -) -ENGINE = MergeTree -ORDER BY id; -``` - -Annoy currently supports `L2Distance` and `cosineDistance` as distance functions. - -Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many nodes are inspected during SELECTs. It can be used to +Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. It can be used to balance runtime and accuracy at runtime. Example: From 4da37a731962ab452ef6a7fb75025f0a6f4e2a51 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:19:50 -0300 Subject: [PATCH 085/159] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 60 ++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 65c43ab04c0..9aaa35dc6d8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -5,7 +5,8 @@ sidebar_position: 106 # argMax -Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. +Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. +Both parts the `arg` and the `max` behave as aggregate functions, they skip `Null` during processing and return not-Null values if not-Null values are available. **Syntax** @@ -49,3 +50,60 @@ Result: │ director │ └──────────────────────┘ ``` + +**Extended example** + +```sql +CREATE TABLE test +( + a Nullable(String), + b Nullable(Int64) +) +ENGINE = Memory AS +SELECT * +FROM values(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL)); + +select * from test; +┌─a────┬────b─┐ +│ a │ 1 │ +│ b │ 2 │ +│ c │ 2 │ +│ ᴺᵁᴸᴸ │ 3 │ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ d │ ᴺᵁᴸᴸ │ +└──────┴──────┘ + +select argMax(a, b), max(b) from test; +┌─argMax(a, b)─┬─max(b)─┐ +│ b │ 3 │ -- argMax = b because it the first not-Null value, max(b) is from another row! +└──────────────┴────────┘ + +select argMax(tuple(a), b) from test; +┌─argMax(tuple(a), b)─┐ +│ (NULL) │ -- Tuple allows to get Null value. +└─────────────────────┘ + +select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; +┌─argMaxA─┬─argMaxB─┐ +│ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +└─────────┴─────────┘ + +select argMax(a, b), max(b) from test where a is Null and b is Null; +┌─argMax(a, b)─┬─max(b)─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +└──────────────┴────────┘ + +select argMax(a, (b,a)) from test; +┌─argMax(a, tuple(b, a))─┐ +│ c │ -- There are two rows with b=2, Tuple in the `Max` allows to get not the first `arg` +└────────────────────────┘ + +select argMax(a, tuple(b)) from test; +┌─argMax(a, tuple(b))─┐ +│ b │ -- Tuple can be used `Max` to not skip Nulls in `Max` +└─────────────────────┘ +``` + +**See also** + +- [Tuple](../../sql-reference/data-types/tuple.md) From 6741a6d7c81ab9a17042170c182327295a6de356 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:21:09 -0300 Subject: [PATCH 086/159] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 9aaa35dc6d8..7800e90eec7 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -106,4 +106,4 @@ select argMax(a, tuple(b)) from test; **See also** -- [Tuple](../../sql-reference/data-types/tuple.md) +- [Tuple](../../data-types/tuple.md) From 113ce8c7574f3e8348cf3ba7981e9a837460ff8b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:43:07 -0300 Subject: [PATCH 087/159] Update argmin.md --- .../aggregate-functions/reference/argmin.md | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index a7c21e3f15b..7972bdf84b8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,6 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. +Both parts the `arg` and the `min` behave as [aggregate functions](../aggregate-functions/index.md), they both [skip `Null`](../aggregate-functions/index.md#null-processing) during processing and return not-Null values if not-Null values are available. **Syntax** @@ -49,3 +50,65 @@ Result: │ worker │ └──────────────────────┘ ``` + +**Extended example** + +```sql +CREATE TABLE test +( + a Nullable(String), + b Nullable(Int64) +) +ENGINE = Memory AS +SELECT * +FROM values((NULL, 0), ('a', 1), ('b', 2), ('c', 2), (NULL, NULL), ('d', NULL)); + +select * from test; +┌─a────┬────b─┐ +│ ᴺᵁᴸᴸ │ 0 │ +│ a │ 1 │ +│ b │ 2 │ +│ c │ 2 │ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +│ d │ ᴺᵁᴸᴸ │ +└──────┴──────┘ + +select argMin(a, b), min(b) from test; +┌─argMin(a, b)─┬─min(b)─┐ +│ a │ 0 │ -- argMin = a because it the first not-Null value, min(b) is from another row! +└──────────────┴────────┘ + +select argMin(tuple(a), b) from test; +┌─argMin(tuple(a), b)─┐ +│ (NULL) │ -- Tuple allows to get Null value. +└─────────────────────┘ + +select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; +┌─argMinA─┬─argMinB─┐ +│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +└─────────┴─────────┘ + +select argMin(a, b), min(b) from test where a is Null and b is Null; +┌─argMin(a, b)─┬─min(b)─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +└──────────────┴────────┘ + +select argMin(a, (b, a)), min(tuple(b, a)) from test; +┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ +│ d │ (NULL,NULL) │ 'd' is the first Not null value for the min +└────────────────────────┴──────────────────┘ + +select argMin((a, b), (b, a)), min(tuple(b, a)) from test; +┌─argMin(tuple(a, b), tuple(b, a))─┬─min(tuple(b, a))─┐ +│ (NULL,NULL) │ (NULL,NULL) │ +└──────────────────────────────────┴──────────────────┘ + +select argMin(a, tuple(b)) from test; +┌─argMax(a, tuple(b))─┐ +│ b │ -- Tuple can be used in `Min` to not skip Nulls in `Min` +└─────────────────────┘ +``` + +**See also** + +- [Tuple](../../data-types/tuple.md) From 40986539775f5d74659521000fe4aa05ca47b06b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:43:47 -0300 Subject: [PATCH 088/159] Update argmin.md --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 7972bdf84b8..a481157784b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../aggregate-functions/index.md), they both [skip `Null`](../aggregate-functions/index.md#null-processing) during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. **Syntax** From a175e4628e640376bc49ceabcebb278159e6d55f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:44:42 -0300 Subject: [PATCH 089/159] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 7800e90eec7..a736804c8dc 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -6,7 +6,7 @@ sidebar_position: 106 # argMax Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as aggregate functions, they skip `Null` during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. **Syntax** @@ -100,7 +100,7 @@ select argMax(a, (b,a)) from test; select argMax(a, tuple(b)) from test; ┌─argMax(a, tuple(b))─┐ -│ b │ -- Tuple can be used `Max` to not skip Nulls in `Max` +│ b │ -- Tuple can be used in `Max` to not skip Nulls in `Max` └─────────────────────┘ ``` From 3e444790af079c3d486ac943b29d4900f4d0576f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:52:51 -0300 Subject: [PATCH 090/159] Update index.md --- docs/en/sql-reference/aggregate-functions/index.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 8951ac4ee6a..019e1cab873 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -72,3 +72,15 @@ FROM t_null_big │ 2.3333333333333335 │ 1.4 │ └────────────────────┴─────────────────────┘ ``` + +Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. + +```sql +select groupArray(b), groupArray(tuple(b)) from t_null_big; +┌─groupArray(b)─┬─groupArray(tuple(b))────────┐ +│ [2,2,3] │ [(2),(NULL),(2),(3),(NULL)] │ +└───────────────┴─────────────────────────────┘ +``` + + + From b4c0d68d0f7a17702d074a9a2f216bbda524c94d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:53:37 -0300 Subject: [PATCH 091/159] Update index.md --- docs/en/sql-reference/aggregate-functions/index.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 019e1cab873..a3808335168 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -76,10 +76,11 @@ FROM t_null_big Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. ```sql -select groupArray(b), groupArray(tuple(b)) from t_null_big; -┌─groupArray(b)─┬─groupArray(tuple(b))────────┐ -│ [2,2,3] │ [(2),(NULL),(2),(3),(NULL)] │ -└───────────────┴─────────────────────────────┘ +select groupArray(b), groupArray(tuple(b)).1 from t_null_big; + +┌─groupArray(b)─┬─tupleElement(groupArray(tuple(b)), 1)─┐ +│ [2,2,3] │ [2,NULL,2,3,NULL] │ +└───────────────┴───────────────────────────────────────┘ ``` From e37cd36db7316c5d2a90df8ca8d8bee8fa016e4a Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:57:28 -0300 Subject: [PATCH 092/159] Update argmin.md --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index a481157784b..067c81f56cf 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -85,7 +85,7 @@ select argMin(tuple(a), b) from test; select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; ┌─argMinA─┬─argMinB─┐ -│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ select argMin(a, b), min(b) from test where a is Null and b is Null; @@ -95,7 +95,7 @@ select argMin(a, b), min(b) from test where a is Null and b is Null; select argMin(a, (b, a)), min(tuple(b, a)) from test; ┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ -│ d │ (NULL,NULL) │ 'd' is the first Not null value for the min +│ d │ (NULL,NULL) │ -- 'd' is the first Not null value for the min └────────────────────────┴──────────────────┘ select argMin((a, b), (b, a)), min(tuple(b, a)) from test; From 15fcad190933aa7b885bdeacb9b2f277104433ac Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 5 Jun 2023 16:57:42 -0300 Subject: [PATCH 093/159] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index a736804c8dc..8a84f361589 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -85,7 +85,7 @@ select argMax(tuple(a), b) from test; select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; ┌─argMaxA─┬─argMaxB─┐ -│ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*) ) columns for the according max(b) +│ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ select argMax(a, b), max(b) from test where a is Null and b is Null; From 3023eb73d02fe7825626a1b5767827432f1513c1 Mon Sep 17 00:00:00 2001 From: Misz606 <113922942+Misz606@users.noreply.github.com> Date: Mon, 5 Jun 2023 21:02:27 +0100 Subject: [PATCH 094/159] Update aggregatingmergetree.md Grammatical update in docs --- .../table-engines/mergetree-family/aggregatingmergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index 2b8b43802ea..62191d9b5e4 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -109,7 +109,7 @@ INSERT INTO test.visits (StartDate, CounterID, Sign, UserID) VALUES (1667446031, 1, 6, 3) ``` -The data are inserted in both the table and the materialized view `test.mv_visits`. +The data is inserted in both the table and the materialized view `test.mv_visits`. To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the materialized view `test.mv_visits`: From 0bc31a72888da7ccf50caee845abef4141077220 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Jun 2023 07:17:30 +0000 Subject: [PATCH 095/159] Fix jepsen runs in PRs --- tests/ci/jepsen_check.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 9d35d2d6e35..c21fafa2605 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -25,6 +25,7 @@ from stopwatch import Stopwatch from tee_popen import TeePopen from upload_result_helper import upload_results from version_helper import get_version_from_repo +from build_check import get_release_or_pr JEPSEN_GROUP_NAME = "jepsen_group" @@ -210,12 +211,7 @@ if __name__ == "__main__": build_name = get_build_name_for_check(check_name) - if pr_info.number == 0: - version = get_version_from_repo() - release_or_pr = f"{version.major}.{version.minor}" - else: - # PR number for anything else - release_or_pr = str(pr_info.number) + release_or_pr, _ = get_release_or_pr(pr_info, get_version_from_repo()) # This check run separately from other checks because it requires exclusive # run (see .github/workflows/jepsen.yml) So we cannot add explicit From ce8b39487e5416b018ce9d03feef4a8114e04f9b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 09:55:50 +0200 Subject: [PATCH 096/159] Update docs/en/engines/table-engines/mergetree-family/annindexes.md Co-authored-by: Nikita Taranov --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 58655c11321..f600f9a015c 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -54,7 +54,7 @@ ENGINE = MergeTree ORDER BY id; ``` -ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively comparatively many more read requests than write requests. +ANN indexes are built during column insertion and merge and `INSERT` and `OPTIMIZE` statements will be slower than for ordinary tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively there are much more read requests than write requests. Similar to regular skip indexes, ANN indexes are constructed over granules and each indexed block consists of `GRANULARITY = `-many granules. For example, if the primary index granularity of the table is 8192 (setting `index_granularity = 8192`) and `GRANULARITY = 2`, From 0b18b75bec6f29e687b6d9cce91fc2cc6c906221 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 08:06:03 +0000 Subject: [PATCH 097/159] Cosmetics: Use abbreviated syntax to read settings --- .../MergeTree/ApproximateNearestNeighborIndexesCommon.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp index c47e53788a7..bf277c55863 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.cpp @@ -53,8 +53,8 @@ ApproximateNearestNeighborInformation::Metric stringToMetric(std::string_view me ApproximateNearestNeighborCondition::ApproximateNearestNeighborCondition(const SelectQueryInfo & query_info, ContextPtr context) : block_with_constants(KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)) - , index_granularity(context->getMergeTreeSettings().get("index_granularity").get()) - , max_limit_for_ann_queries(context->getSettings().get("max_limit_for_ann_queries").get()) + , index_granularity(context->getMergeTreeSettings().index_granularity) + , max_limit_for_ann_queries(context->getSettings().max_limit_for_ann_queries) , index_is_useful(checkQueryStructure(query_info)) {} diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 72dd92ead5e..0a2df639b69 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -210,7 +210,7 @@ MergeTreeIndexConditionAnnoy::MergeTreeIndexConditionAnnoy( ContextPtr context) : ann_condition(query, context) , distance_function(distance_function_) - , search_k(context->getSettings().get("annoy_index_search_k_nodes").get()) + , search_k(context->getSettings().annoy_index_search_k_nodes) {} bool MergeTreeIndexConditionAnnoy::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const From 2c5c0c5c9f82963903a618e7686c39a59899f020 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 08:14:50 +0000 Subject: [PATCH 098/159] Cosmetics: Remove exception path for something checked elsewhere already --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 0a2df639b69..b15b1bb1a91 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -229,8 +229,7 @@ std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndex return getUsefulRangesImpl(idx_granule); else if (distance_function == "cosineDistance") return getUsefulRangesImpl(idx_granule); - else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown distance name. Must be 'L2Distance' or 'cosineDistance'. Got {}", distance_function); + std::unreachable(); } template From a96c1ea86f1fa662e8e775faeaf4b3a8053c6eb2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 6 Jun 2023 15:15:51 +0200 Subject: [PATCH 099/159] Fix --- src/Interpreters/Cache/FileSegment.cpp | 2 +- src/Interpreters/Cache/Metadata.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 1b7fe54a769..7b82c58080c 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -381,7 +381,7 @@ void FileSegment::write(const char * from, size_t size, size_t offset) const auto file_size = fs::file_size(file_segment_path); chassert(downloaded_size <= file_size); chassert(reserved_size >= file_size); - chassert(file_size <= range().right + 1); + chassert(file_size <= range().size()); if (downloaded_size != file_size) downloaded_size = file_size; } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 9dff77e2af8..4794d1b3264 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -370,8 +370,8 @@ KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset, const FileSegm bool exists = fs::exists(path); if (exists) { - LOG_TEST(log, "Removed file segment at path: {}", path); fs::remove(path); + LOG_TEST(log, "Removed file segment at path: {}", path); } else if (file_segment->downloaded_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected path {} to exist"); From 9ae4d929eacac5ae4681cce91ec2f65b198cf3c9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 6 Jun 2023 15:16:18 +0200 Subject: [PATCH 100/159] Update src/Interpreters/tests/gtest_convertFieldToType.cpp --- src/Interpreters/tests/gtest_convertFieldToType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp index 5421c192ac7..f4de36cbecc 100644 --- a/src/Interpreters/tests/gtest_convertFieldToType.cpp +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -56,7 +56,7 @@ TEST_P(ConvertFieldToTypeTest, convert) } } -// Basically nuber of seconds in a day, works for UTC here +// Basically, the number of seconds in a day works for UTC here const long long int Day = 24 * 60 * 60; // 123 is arbitrary value here From 4ca902ddddc36dbe54b7a6e0f4a7e8185d0b35ca Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:16:56 -0300 Subject: [PATCH 101/159] Update docs/en/sql-reference/aggregate-functions/index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/index.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index a3808335168..050142fdd99 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -76,9 +76,12 @@ FROM t_null_big Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. ```sql -select groupArray(b), groupArray(tuple(b)).1 from t_null_big; +SELECT + groupArray(y), + groupArray(tuple(y)).1 +FROM t_null_big; -┌─groupArray(b)─┬─tupleElement(groupArray(tuple(b)), 1)─┐ +┌─groupArray(y)─┬─tupleElement(groupArray(tuple(y)), 1)─┐ │ [2,2,3] │ [2,NULL,2,3,NULL] │ └───────────────┴───────────────────────────────────────┘ ``` From 5d7b8b3d13502a2eb842dfbfe3b1b704b638badb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:04 -0300 Subject: [PATCH 102/159] Update docs/en/sql-reference/aggregate-functions/index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/index.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 050142fdd99..25f9d05a76c 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -85,6 +85,3 @@ FROM t_null_big; │ [2,2,3] │ [2,NULL,2,3,NULL] │ └───────────────┴───────────────────────────────────────┘ ``` - - - From 2a9792cebd6e15818a65bd6566cf683bcb623225 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:21 -0300 Subject: [PATCH 103/159] Update docs/en/sql-reference/aggregate-functions/reference/argmax.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 8a84f361589..5e80131df4c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -90,7 +90,7 @@ select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; select argMax(a, b), max(b) from test where a is Null and b is Null; ┌─argMax(a, b)─┬─max(b)─┐ -│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ select argMax(a, (b,a)) from test; From cda9535554fa28c76d11115fdf276c25e0449f04 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:46 -0300 Subject: [PATCH 104/159] Update docs/en/sql-reference/aggregate-functions/index.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index 25f9d05a76c..ea270e83a3c 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -73,7 +73,7 @@ FROM t_null_big └────────────────────┴─────────────────────┘ ``` -Also you can use [Tuple](../data-types/tuple.md) to change NULL skipping behavior. +Also you can use [Tuple](../data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. ```sql SELECT From 1a517bb332f01da3b5b64dadbb481339b6bdf7ac Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:17:55 -0300 Subject: [PATCH 105/159] Update docs/en/sql-reference/aggregate-functions/reference/argmax.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 5e80131df4c..76b9e206abe 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -80,7 +80,7 @@ select argMax(a, b), max(b) from test; select argMax(tuple(a), b) from test; ┌─argMax(tuple(a), b)─┐ -│ (NULL) │ -- Tuple allows to get Null value. +│ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; From 5308abb93a22cf19d10078e533a3b09741a713f7 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:18:13 -0300 Subject: [PATCH 106/159] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 067c81f56cf..304abf512da 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -90,7 +90,7 @@ select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; select argMin(a, b), min(b) from test where a is Null and b is Null; ┌─argMin(a, b)─┬─min(b)─┐ -│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- Nulls are not skipped because only Null values are available +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- ll aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ select argMin(a, (b, a)), min(tuple(b, a)) from test; From f992d10ae7af2cd7d37da65055be4f26fdfa9957 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:18:22 -0300 Subject: [PATCH 107/159] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 304abf512da..816dd9fcac1 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -75,7 +75,7 @@ select * from test; select argMin(a, b), min(b) from test; ┌─argMin(a, b)─┬─min(b)─┐ -│ a │ 0 │ -- argMin = a because it the first not-Null value, min(b) is from another row! +│ a │ 0 │ -- argMin = a because it the first not Null value, min(b) is from another row! └──────────────┴────────┘ select argMin(tuple(a), b) from test; From 41be2fcbc7bd2227df9c506d2cbde82ded06e690 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:18:34 -0300 Subject: [PATCH 108/159] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 816dd9fcac1..df4b28b070c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -95,7 +95,7 @@ select argMin(a, b), min(b) from test where a is Null and b is Null; select argMin(a, (b, a)), min(tuple(b, a)) from test; ┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ -│ d │ (NULL,NULL) │ -- 'd' is the first Not null value for the min +│ d │ (NULL,NULL) │ -- 'd' is the first not Null value for the min └────────────────────────┴──────────────────┘ select argMin((a, b), (b, a)), min(tuple(b, a)) from test; From 834918dc1cef6af6eca60d57953a82b982f993aa Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:19:08 -0300 Subject: [PATCH 109/159] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index df4b28b070c..4db656f7e54 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -105,7 +105,7 @@ select argMin((a, b), (b, a)), min(tuple(b, a)) from test; select argMin(a, tuple(b)) from test; ┌─argMax(a, tuple(b))─┐ -│ b │ -- Tuple can be used in `Min` to not skip Nulls in `Min` +│ d │ -- Tuple can be used in `min` to not skip rows with Null values as b. └─────────────────────┘ ``` From 9a1f6ac719b0018aa050c0c8ad099bcd65111721 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:20:42 -0300 Subject: [PATCH 110/159] Update argmax.md --- docs/en/sql-reference/aggregate-functions/reference/argmax.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 76b9e206abe..91b85bce2ff 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -6,7 +6,7 @@ sidebar_position: 106 # argMax Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. **Syntax** From d6ee50577a5d77887612a74bd43f7bd66a10e666 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:25:32 -0300 Subject: [PATCH 111/159] Update argmin.md --- .../sql-reference/aggregate-functions/reference/argmin.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 4db656f7e54..a5208f11de6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not-Null values if not-Null values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. **Syntax** @@ -90,7 +90,7 @@ select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; select argMin(a, b), min(b) from test where a is Null and b is Null; ┌─argMin(a, b)─┬─min(b)─┐ -│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- ll aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ select argMin(a, (b, a)), min(tuple(b, a)) from test; @@ -100,7 +100,7 @@ select argMin(a, (b, a)), min(tuple(b, a)) from test; select argMin((a, b), (b, a)), min(tuple(b, a)) from test; ┌─argMin(tuple(a, b), tuple(b, a))─┬─min(tuple(b, a))─┐ -│ (NULL,NULL) │ (NULL,NULL) │ +│ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because Tuple allows to don't skip Nulls and min(tuple(b, a)) in this case is minimal value for this dataset └──────────────────────────────────┴──────────────────┘ select argMin(a, tuple(b)) from test; From 496bc25bff11c023ecc7f05420889538ff59779e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:26:35 -0300 Subject: [PATCH 112/159] Update docs/en/sql-reference/aggregate-functions/reference/argmin.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index a5208f11de6..3787c028564 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -80,7 +80,7 @@ select argMin(a, b), min(b) from test; select argMin(tuple(a), b) from test; ┌─argMin(tuple(a), b)─┐ -│ (NULL) │ -- Tuple allows to get Null value. +│ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; From e63fc91e73a5d718f3daf6383114978ce93a3b40 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:30:10 -0300 Subject: [PATCH 113/159] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 91b85bce2ff..93e1fac6d67 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -6,7 +6,7 @@ sidebar_position: 106 # argMax Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. +Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -61,7 +61,7 @@ CREATE TABLE test ) ENGINE = Memory AS SELECT * -FROM values(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL)); +FROM VALUES(('a', 1), ('b', 2), ('c', 2), (NULL, 3), (NULL, NULL), ('d', NULL)); select * from test; ┌─a────┬────b─┐ @@ -73,34 +73,34 @@ select * from test; │ d │ ᴺᵁᴸᴸ │ └──────┴──────┘ -select argMax(a, b), max(b) from test; +SELECT argMax(a, b), max(b) FROM test; ┌─argMax(a, b)─┬─max(b)─┐ -│ b │ 3 │ -- argMax = b because it the first not-Null value, max(b) is from another row! +│ b │ 3 │ -- argMax = 'b' because it the first not Null value, max(b) is from another row! └──────────────┴────────┘ -select argMax(tuple(a), b) from test; +SELECT argMax(tuple(a), b) FROM test; ┌─argMax(tuple(a), b)─┐ │ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ -select (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB from test; +SELECT (argMax((a, b), b) as t).1 argMaxA, t.2 argMaxB FROM test; ┌─argMaxA─┬─argMaxB─┐ │ ᴺᵁᴸᴸ │ 3 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ -select argMax(a, b), max(b) from test where a is Null and b is Null; +SELECT argMax(a, b), max(b) FROM test WHERE a IS NULL AND b IS NULL; ┌─argMax(a, b)─┬─max(b)─┐ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ -select argMax(a, (b,a)) from test; +SELECT argMax(a, (b,a)) FROM test; ┌─argMax(a, tuple(b, a))─┐ -│ c │ -- There are two rows with b=2, Tuple in the `Max` allows to get not the first `arg` +│ c │ -- There are two rows with b=2, `Tuple` in the `Max` allows to get not the first `arg` └────────────────────────┘ -select argMax(a, tuple(b)) from test; +SELECT argMax(a, tuple(b)) FROM test; ┌─argMax(a, tuple(b))─┐ -│ b │ -- Tuple can be used in `Max` to not skip Nulls in `Max` +│ b │ -- `Tuple` can be used in `Max` to not skip Nulls in `Max` └─────────────────────┘ ``` From 67a6623a3896b4deae5d69cdf84742da5b24da3b Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 6 Jun 2023 10:32:10 -0300 Subject: [PATCH 114/159] Update argmin.md --- .../aggregate-functions/reference/argmin.md | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 3787c028564..4e549e5b04c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not Null values if not Null values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -61,7 +61,7 @@ CREATE TABLE test ) ENGINE = Memory AS SELECT * -FROM values((NULL, 0), ('a', 1), ('b', 2), ('c', 2), (NULL, NULL), ('d', NULL)); +FROM VALUES((NULL, 0), ('a', 1), ('b', 2), ('c', 2), (NULL, NULL), ('d', NULL)); select * from test; ┌─a────┬────b─┐ @@ -73,39 +73,39 @@ select * from test; │ d │ ᴺᵁᴸᴸ │ └──────┴──────┘ -select argMin(a, b), min(b) from test; +SELECT argMin(a, b), min(b) FROM test; ┌─argMin(a, b)─┬─min(b)─┐ -│ a │ 0 │ -- argMin = a because it the first not Null value, min(b) is from another row! +│ a │ 0 │ -- argMin = a because it the first not `NULL` value, min(b) is from another row! └──────────────┴────────┘ -select argMin(tuple(a), b) from test; +SELECT argMin(tuple(a), b) FROM test; ┌─argMin(tuple(a), b)─┐ │ (NULL) │ -- The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value └─────────────────────┘ -select (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; +SELECT (argMin((a, b), b) as t).1 argMinA, t.2 argMinB from test; ┌─argMinA─┬─argMinB─┐ -│ ᴺᵁᴸᴸ │ 0 │ -- you can use Tuple and get both (all - tuple(*)) columns for the according max(b) +│ ᴺᵁᴸᴸ │ 0 │ -- you can use `Tuple` and get both (all - tuple(*)) columns for the according max(b) └─────────┴─────────┘ -select argMin(a, b), min(b) from test where a is Null and b is Null; +SELECT argMin(a, b), min(b) FROM test WHERE a IS NULL and b IS NULL; ┌─argMin(a, b)─┬─min(b)─┐ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -- All aggregated rows contains at least one `NULL` value because of the filter, so all rows are skipped, therefore the result will be `NULL` └──────────────┴────────┘ -select argMin(a, (b, a)), min(tuple(b, a)) from test; +SELECT argMin(a, (b, a)), min(tuple(b, a)) FROM test; ┌─argMin(a, tuple(b, a))─┬─min(tuple(b, a))─┐ -│ d │ (NULL,NULL) │ -- 'd' is the first not Null value for the min +│ d │ (NULL,NULL) │ -- 'd' is the first not `NULL` value for the min └────────────────────────┴──────────────────┘ -select argMin((a, b), (b, a)), min(tuple(b, a)) from test; +SELECT argMin((a, b), (b, a)), min(tuple(b, a)) FROM test; ┌─argMin(tuple(a, b), tuple(b, a))─┬─min(tuple(b, a))─┐ -│ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because Tuple allows to don't skip Nulls and min(tuple(b, a)) in this case is minimal value for this dataset +│ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because `Tuple` allows to don't skip `NULL` and min(tuple(b, a)) in this case is minimal value for this dataset └──────────────────────────────────┴──────────────────┘ select argMin(a, tuple(b)) from test; ┌─argMax(a, tuple(b))─┐ -│ d │ -- Tuple can be used in `min` to not skip rows with Null values as b. +│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b. └─────────────────────┘ ``` From 68a9ea72dee4ff94ac061544f366ce9a10fc2053 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Jun 2023 16:10:01 +0200 Subject: [PATCH 115/159] Revert "Merge pull request #50307 from ZhiguoZh/20230527-toyyyymm" This reverts commit 17261e52da18a739b59895e9cd97dff3e72b66b8, reversing changes made to d302eae85efc7f4c1d29993484c7d8b068e1be62. --- .../OptimizeDateFilterVisitor.cpp | 33 ++------- ..._date_filter_predicate_optimizer.reference | 69 ------------------- .../02764_date_filter_predicate_optimizer.sql | 23 ------- 3 files changed, 5 insertions(+), 120 deletions(-) diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp index aec2dec19c8..58e1b3335f9 100644 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ b/src/Interpreters/OptimizeDateFilterVisitor.cpp @@ -10,37 +10,14 @@ namespace DB { -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 compare_to) +ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) { const DateLUTImpl & date_lut = DateLUT::instance(); - String start_date; - String end_date; + if (converter != "toYear") return {}; - if (converter == "toYear") - { - UInt64 year = compare_to; - start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); - } - else if (converter == "toYYYYMM") - { - UInt64 year = compare_to / 100; - UInt64 month = compare_to % 100; - - if (month == 0 || month > 12) return {}; - - static constexpr UInt8 days_of_month[] = {31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; - - bool leap_year = (year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)); - - start_date = date_lut.dateToString(date_lut.makeDayNum(year, month, 1)); - end_date = date_lut.dateToString(date_lut.makeDayNum(year, month, days_of_month[month - 1] + (leap_year && month == 2))); - } - else - { - return {}; - } + String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); + String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); if (comparator == "equals") { @@ -105,7 +82,7 @@ bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) { if (const auto * func = function.arguments->children[i]->as(); func) { - if (func->name == "toYear" || func->name == "toYYYYMM") + if (func->name == "toYear") { func_id = i; } diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference index 54704fb3b3e..e5c608ddc1a 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference @@ -37,72 +37,3 @@ WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) A SELECT value1 FROM t WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) -SELECT value1 -FROM t -WHERE ((date1 >= \'1900-02-01\') AND (date1 <= \'1900-02-28\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1992-02-01\') AND (date1 <= \'1992-02-29\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'2000-02-01\') AND (date1 <= \'2000-02-29\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-01-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-02-01\') AND (date1 <= \'1993-02-28\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-03-01\') AND (date1 <= \'1993-03-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-04-01\') AND (date1 <= \'1993-04-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-05-01\') AND (date1 <= \'1993-05-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-06-01\') AND (date1 <= \'1993-06-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-07-01\') AND (date1 <= \'1993-07-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-08-01\') AND (date1 <= \'1993-08-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-09-01\') AND (date1 <= \'1993-09-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-10-01\') AND (date1 <= \'1993-10-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-11-01\') AND (date1 <= \'1993-11-30\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-12-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 < \'1992-03-01\') OR (date1 > \'1992-03-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 > \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 <= \'1992-03-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\'))) AND ((id >= 1) AND (id <= 3)) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql index a04273bbb18..563468d4f82 100644 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql +++ b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql @@ -13,28 +13,5 @@ EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 A EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 190002 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199202 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 200002 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199301 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199302 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199303 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199304 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199305 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199306 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199307 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199308 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199309 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199310 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199311 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; DROP TABLE t; From 8d2b8683125b3ba1e2ccddff078c3af9820a19af Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Jun 2023 16:10:22 +0200 Subject: [PATCH 116/159] Revert "Merge pull request #50062 from ZhiguoZh/20230511-toyear" This reverts commit 55c2dbcc2d1068dae78e7be0929b193edb23d75c, reversing changes made to 23f894b995feb4d0045ba24593bd457e39b7d11d. --- .../OptimizeDateFilterVisitor.cpp | 121 ------------------ src/Interpreters/OptimizeDateFilterVisitor.h | 20 --- src/Interpreters/TreeOptimizer.cpp | 19 --- ..._date_filter_predicate_optimizer.reference | 39 ------ .../02764_date_filter_predicate_optimizer.sql | 17 --- 5 files changed, 216 deletions(-) delete mode 100644 src/Interpreters/OptimizeDateFilterVisitor.cpp delete mode 100644 src/Interpreters/OptimizeDateFilterVisitor.h delete mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference delete mode 100644 tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql diff --git a/src/Interpreters/OptimizeDateFilterVisitor.cpp b/src/Interpreters/OptimizeDateFilterVisitor.cpp deleted file mode 100644 index 58e1b3335f9..00000000000 --- a/src/Interpreters/OptimizeDateFilterVisitor.cpp +++ /dev/null @@ -1,121 +0,0 @@ -#include - -#include -#include -#include -#include -#include - - -namespace DB -{ - -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const String & converter, const String & column, UInt64 year) -{ - const DateLUTImpl & date_lut = DateLUT::instance(); - - if (converter != "toYear") return {}; - - String start_date = date_lut.dateToString(date_lut.makeDayNum(year, 1, 1)); - String end_date = date_lut.dateToString(date_lut.makeDayNum(year, 12, 31)); - - if (comparator == "equals") - { - return makeASTFunction("and", - makeASTFunction("greaterOrEquals", - std::make_shared(column), - std::make_shared(start_date) - ), - makeASTFunction("lessOrEquals", - std::make_shared(column), - std::make_shared(end_date) - ) - ); - } - else if (comparator == "notEquals") - { - return makeASTFunction("or", - makeASTFunction("less", - std::make_shared(column), - std::make_shared(start_date) - ), - makeASTFunction("greater", - std::make_shared(column), - std::make_shared(end_date) - ) - ); - } - else if (comparator == "less" || comparator == "greaterOrEquals") - { - return makeASTFunction(comparator, - std::make_shared(column), - std::make_shared(start_date) - ); - } - else - { - return makeASTFunction(comparator, - std::make_shared(column), - std::make_shared(end_date) - ); - } -} - -bool rewritePredicateInPlace(ASTFunction & function, ASTPtr & ast) -{ - const static std::unordered_map swap_relations = { - {"equals", "equals"}, - {"notEquals", "notEquals"}, - {"less", "greater"}, - {"greater", "less"}, - {"lessOrEquals", "greaterOrEquals"}, - {"greaterOrEquals", "lessOrEquals"}, - }; - - if (!swap_relations.contains(function.name)) return false; - - if (!function.arguments || function.arguments->children.size() != 2) return false; - - size_t func_id = function.arguments->children.size(); - - for (size_t i = 0; i < function.arguments->children.size(); i++) - { - if (const auto * func = function.arguments->children[i]->as(); func) - { - if (func->name == "toYear") - { - func_id = i; - } - } - } - - if (func_id == function.arguments->children.size()) return false; - - size_t literal_id = 1 - func_id; - const auto * literal = function.arguments->children[literal_id]->as(); - - if (!literal || literal->value.getType() != Field::Types::UInt64) return false; - - UInt64 compare_to = literal->value.get(); - String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); - - const auto * func = function.arguments->children[func_id]->as(); - const auto * column_id = func->arguments->children.at(0)->as(); - - if (!column_id) return false; - - String column = column_id->name(); - - const auto new_ast = generateOptimizedDateFilterAST(comparator, func->name, column, compare_to); - - if (!new_ast) return false; - - ast = new_ast; - return true; -} - -void OptimizeDateFilterInPlaceData::visit(ASTFunction & function, ASTPtr & ast) const -{ - rewritePredicateInPlace(function, ast); -} -} diff --git a/src/Interpreters/OptimizeDateFilterVisitor.h b/src/Interpreters/OptimizeDateFilterVisitor.h deleted file mode 100644 index 84394372901..00000000000 --- a/src/Interpreters/OptimizeDateFilterVisitor.h +++ /dev/null @@ -1,20 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class ASTFunction; - -/// Rewrite the predicates in place -class OptimizeDateFilterInPlaceData -{ -public: - using TypeToVisit = ASTFunction; - void visit(ASTFunction & function, ASTPtr & ast) const; -}; - -using OptimizeDateFilterInPlaceMatcher = OneTypeMatcher; -using OptimizeDateFilterInPlaceVisitor = InDepthNodeVisitor; -} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 825114b20b7..c38b3c79026 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -678,21 +677,6 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) RemoveInjectiveFunctionsVisitor(data).visit(query); } -void optimizeDateFilters(ASTSelectQuery * select_query) -{ - /// Predicates in HAVING clause has been moved to WHERE clause. - if (select_query->where()) - { - OptimizeDateFilterInPlaceVisitor::Data data; - OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refWhere()); - } - if (select_query->prewhere()) - { - OptimizeDateFilterInPlaceVisitor::Data data; - OptimizeDateFilterInPlaceVisitor(data).visit(select_query->refPrewhere()); - } -} - void transformIfStringsIntoEnum(ASTPtr & query) { std::unordered_set function_names = {"if", "transform"}; @@ -796,9 +780,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, tables_with_columns, result.storage_snapshot->metadata, result.storage); } - /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, toISOWeek, etc. - optimizeDateFilters(select_query); - /// GROUP BY injective function elimination. optimizeGroupBy(select_query, context); diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference deleted file mode 100644 index e5c608ddc1a..00000000000 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.reference +++ /dev/null @@ -1,39 +0,0 @@ -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 < \'1993-01-01\') OR (date1 > \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 > \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 <= \'1993-12-31\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1997-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) OR ((date1 >= \'1994-01-01\') AND (date1 <= \'1994-12-31\'))) AND ((id >= 1) AND (id <= 3)) -SELECT - value1, - toYear(date1) AS year1 -FROM t -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -PREWHERE (date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\') -WHERE ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM t -WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 <= \'1993-12-31\')) diff --git a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql b/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql deleted file mode 100644 index 563468d4f82..00000000000 --- a/tests/queries/0_stateless/02764_date_filter_predicate_optimizer.sql +++ /dev/null @@ -1,17 +0,0 @@ -DROP TABLE IF EXISTS t; -CREATE TABLE t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; - -DROP TABLE t; From 1910d6580e6a6cd7ad985976ed08885b3b091219 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Jun 2023 16:13:51 +0200 Subject: [PATCH 117/159] Add test for the reverted broken optimizations --- .../02783_date_predicate_optimizations.reference | 2 ++ .../02783_date_predicate_optimizations.sql | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02783_date_predicate_optimizations.reference create mode 100644 tests/queries/0_stateless/02783_date_predicate_optimizations.sql diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference new file mode 100644 index 00000000000..cd689b93034 --- /dev/null +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference @@ -0,0 +1,2 @@ +2021-12-31 23:00:00 0 +2021-12-31 23:00:00 0 diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql new file mode 100644 index 00000000000..abb13f1005e --- /dev/null +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql @@ -0,0 +1,13 @@ +CREATE TABLE source +( + `ts` DateTime('UTC'), + `n` Int32 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(ts) +ORDER BY tuple(); + +INSERT INTO source values ('2021-12-31 23:00:00', 0); + +SELECT * FROM source WHERE toYYYYMM(ts) = 202112; +SELECT * FROM source WHERE toYear(ts) = 2021; From 6c82ee45e2f1886219967dbadcde8e6a59bd84b1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 6 Jun 2023 18:27:16 +0200 Subject: [PATCH 118/159] Fix build --- src/Interpreters/tests/gtest_convertFieldToType.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp index f4de36cbecc..e259418d1c3 100644 --- a/src/Interpreters/tests/gtest_convertFieldToType.cpp +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -9,7 +9,6 @@ #include #include "base/Decimal.h" #include "base/types.h" -#include "gtest/gtest.h" using namespace DB; From 38edd6c3e778aad7a5a8a38c294c0c1e340990a1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 6 Jun 2023 18:28:34 +0200 Subject: [PATCH 119/159] Update src/Interpreters/tests/gtest_convertFieldToType.cpp --- src/Interpreters/tests/gtest_convertFieldToType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/tests/gtest_convertFieldToType.cpp b/src/Interpreters/tests/gtest_convertFieldToType.cpp index e259418d1c3..cda9311dcbe 100644 --- a/src/Interpreters/tests/gtest_convertFieldToType.cpp +++ b/src/Interpreters/tests/gtest_convertFieldToType.cpp @@ -56,7 +56,7 @@ TEST_P(ConvertFieldToTypeTest, convert) } // Basically, the number of seconds in a day works for UTC here -const long long int Day = 24 * 60 * 60; +const Int64 Day = 24 * 60 * 60; // 123 is arbitrary value here From 2a756a7e6c7d564aae6e4895b53e4b2d6f18cfbd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 17:05:23 +0000 Subject: [PATCH 120/159] Cosmetics: Make default tree count / distance function constants --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index b15b1bb1a91..e6c11c839fe 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -314,12 +314,14 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { - uint64_t trees = 100; - String distance_function = "L2Distance"; + static constexpr auto default_trees = 100uz; + static constexpr auto default_distance_function = "L2Distance"; + String distance_function = default_distance_function; if (!index.arguments.empty()) distance_function = index.arguments[0].get(); + uint64_t trees = default_trees; if (index.arguments.size() > 1) trees = index.arguments[1].get(); From a74d3ca3289806eccf6cd66ac29725bc3655bbe8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 17:06:47 +0000 Subject: [PATCH 121/159] Fix style check --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e6c11c839fe..1a28f28f746 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -23,7 +23,6 @@ namespace ErrorCodes extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; } From b6c8ce30ec5d8eeff846b273ed8aa2e6b00241f0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 6 Jun 2023 19:38:11 +0200 Subject: [PATCH 122/159] Disable 01676_clickhouse_client_autocomplete under UBSan --- .../queries/0_stateless/01676_clickhouse_client_autocomplete.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh index 42ae5e84f44..db62dedb5b4 100755 --- a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-ubsan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e9c267ed696f30d440b35ed3bb215d550eb8aec7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 6 Jun 2023 17:51:53 +0000 Subject: [PATCH 123/159] Fix converting Null to LowCardinality(Nullable) in values table function --- src/Interpreters/convertFieldToType.cpp | 2 +- .../0_stateless/02782_values_null_to_lc_nullable.reference | 1 + tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference create mode 100644 tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index dc61e748db6..3e8fab80aaf 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -534,7 +534,7 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_type, const IDataType * from_type_hint) { bool is_null = from_value.isNull(); - if (is_null && !to_type.isNullable()) + if (is_null && !to_type.isNullable() && !to_type.isLowCardinalityNullable()) throw Exception(ErrorCodes::TYPE_MISMATCH, "Cannot convert NULL to {}", to_type.getName()); Field converted = convertFieldToType(from_value, to_type, from_type_hint); diff --git a/tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference new file mode 100644 index 00000000000..dec7d2fabd2 --- /dev/null +++ b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.reference @@ -0,0 +1 @@ +\N diff --git a/tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql new file mode 100644 index 00000000000..250fe6b7551 --- /dev/null +++ b/tests/queries/0_stateless/02782_values_null_to_lc_nullable.sql @@ -0,0 +1,2 @@ +select * from values('s LowCardinality(Nullable(String))', (NULL)); + From 42c054789561920adf7ce4770968ba303a70f244 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 6 Jun 2023 19:25:43 +0000 Subject: [PATCH 124/159] Remove clang-tidy exclude --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 1a28f28f746..ffed9e01df0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -123,7 +123,6 @@ MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( template MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndReset() { - // NOLINTNEXTLINE(*) index->build(static_cast(trees), /*number_of_threads=*/1); auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; From be8e0487994d70c3c6350cb675475b12dffda8e5 Mon Sep 17 00:00:00 2001 From: johanngan Date: Tue, 6 Jun 2023 16:28:44 -0500 Subject: [PATCH 125/159] Revert invalid RegExpTreeDictionary optimization This reverts the following commits: - e77dd810369ad5fcf957393e4fc71a8a6220b04e - e8527e720b2ab12b3327f1e3886aace402a292c6 Additionally, functional tests are added. When scanning complex regexp nodes sequentially with RE2, the old code has an optimization to break out of the loop early upon finding a leaf node that matches. This is an invalid optimization because there's no guarantee that it's actually a VALID match, because its parents might NOT have matched. Semantically, a user would expect this match to be discarded and for the search to continue. Instead, since we skipped matching after the first false positive, subsequent nodes that would have matched are missing from the output value. This affects both dictGet and dictGetAll. It's difficult to distinguish a true positive from a false positive while looping through complex_regexp_nodes because we would have to scan all the parents of a matching node to confirm a true positive. Trying to do this might actually end up being slower than just scanning every complex regexp node, because complex_regexp_nodes is only a subset of all the tree nodes; we may end up duplicating work with scanning that Vectorscan has already done, depending on whether the parent nodes are "simple" or "complex". So instead of trying to fix this optimization, just remove it entirely. --- src/Dictionaries/RegExpTreeDictionary.cpp | 14 ---- ...04_regexp_dictionary_yaml_source.reference | 16 +++++ .../02504_regexp_dictionary_yaml_source.sh | 64 +++++++++++++++++++ 3 files changed, 80 insertions(+), 14 deletions(-) diff --git a/src/Dictionaries/RegExpTreeDictionary.cpp b/src/Dictionaries/RegExpTreeDictionary.cpp index 8d0af9b0abf..3852cca6928 100644 --- a/src/Dictionaries/RegExpTreeDictionary.cpp +++ b/src/Dictionaries/RegExpTreeDictionary.cpp @@ -129,17 +129,6 @@ struct RegExpTreeDictionary::RegexTreeNode return searcher.Match(haystack, 0, size, re2_st::RE2::Anchor::UNANCHORED, nullptr, 0); } - /// check if this node can cover all the attributes from the query. - bool containsAll(const std::unordered_map & matching_attributes) const - { - for (const auto & [key, value] : matching_attributes) - { - if (!attributes.contains(key)) - return false; - } - return true; - } - struct AttributeValue { Field field; @@ -691,9 +680,6 @@ std::unordered_map RegExpTreeDictionary::match( if (node_ptr->match(reinterpret_cast(keys_data.data()) + offset, length)) { match_result.insertNodeID(node_ptr->id); - /// When this node is leaf and contains all the required attributes, it means a match. - if (node_ptr->containsAll(attributes) && node_ptr->children.empty()) - break; } } diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference index 437012dd516..79871e3716c 100644 --- a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference +++ b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.reference @@ -11,3 +11,19 @@ (['ClickHouse Documentation','ClickHouse'],[0,1],['/en'],['ClickHouse']) (['Documentation','GitHub'],[2,3],[NULL],[]) (['Documentation','GitHub'],[2,3],[NULL],[]) +ClickHouse +['ClickHouse'] +ClickHouse Documentation +['ClickHouse Documentation','ClickHouse','Documentation'] +GitHub Documentation +['GitHub Documentation','GitHub'] +Documentation +['Documentation'] +ClickHouse +['ClickHouse'] +ClickHouse Documentation +['ClickHouse Documentation','ClickHouse','Documentation'] +GitHub Documentation +['GitHub Documentation','GitHub'] +Documentation +['Documentation'] diff --git a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh index ac0793460a9..5e8985406ae 100755 --- a/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh +++ b/tests/queries/0_stateless/02504_regexp_dictionary_yaml_source.sh @@ -175,6 +175,70 @@ select dictGetAll('regexp_dict3', ('tag', 'topological_index', 'captured', 'pare select dictGetAll('regexp_dict3', ('tag', 'topological_index', 'captured', 'parent'), 'github.com/clickhouse/tree/master/docs', 2); " +# Test that things work the same for "simple" regexps that go through Hyperscan and "complex" regexps that go through RE2. +# An easy way to force the use of RE2 is to disable Hyperscan. +# This tree is constructed purposely so that text might (falsely) match leaf nodes without matching their corresponding parent nodes +cat > "$yaml" < #include #include +#include #include #include #include @@ -948,25 +949,52 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd std::list useful_indices; std::map, MergedDataSkippingIndexAndCondition> merged_indices; + std::unordered_set ignored_index_names; + + if (use_skip_indexes && settings.ignore_data_skipping_indices.changed) + { + const auto & indices = settings.ignore_data_skipping_indices.toString(); + Tokens tokens(indices.data(), &indices[indices.size()], settings.max_query_size); + IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); + Expected expected; + + /// Use an unordered list rather than string vector + auto parse_single_id_or_literal = [&] + { + String str; + if (!parseIdentifierOrStringLiteral(pos, expected, str)) + return false; + + ignored_index_names.insert(std::move(str)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_single_id_or_literal, false)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); + } if (use_skip_indexes) { for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) - { - auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); - if (inserted) - it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - it->second.addIndex(index_helper); - } - else + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if(!ignored_index_names.contains(index.name)) { - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); + if (inserted) + it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + + it->second.addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + useful_indices.emplace_back(index_helper, condition); + } } } } diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference new file mode 100644 index 00000000000..af1fce5ba13 --- /dev/null +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -0,0 +1,44 @@ +1 2 3 +1 2 3 +1 2 3 +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql new file mode 100644 index 00000000000..ab314ae5ea2 --- /dev/null +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS data_02771; +CREATE TABLE data_02771 +( + key Int, + x Int, + y Int, + INDEX x_idx x TYPE minmax GRANULARITY 1, + INDEX y_idx y TYPE minmax GRANULARITY 1, + INDEX xy_idx (x,y) TYPE minmax GRANULARITY 1 +) +Engine=MergeTree() +ORDER BY key; + +INSERT INTO data_02771 VALUES (1, 2, 3); + +SELECT * FROM data_02771; +SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices=''; -- { serverError 6 } +SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='x_idx'; +SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='na_idx'; + +SELECT * FROM data_02771 WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- { serverError 277 } +SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; +EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2; +EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; + +DROP TABLE data_02771; From f552b96451bd4c826a9e7d1bff669301c3c4bccc Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Mon, 29 May 2023 14:00:00 -0700 Subject: [PATCH 130/159] Add docs for ignore index --- docs/en/operations/settings/settings.md | 83 +++++++++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5730503a670..5b0c6b3c8c2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -227,6 +227,89 @@ SELECT * FROM data_01515 WHERE d1 = 0 SETTINGS force_data_skipping_indices='`d1_ SELECT * FROM data_01515 WHERE d1 = 0 AND assumeNotNull(d1_null) = 0 SETTINGS force_data_skipping_indices='`d1_idx`, d1_null_idx'; -- Ok. ``` +## ignore_data_skipping_indices {#settings-ignore_data_skipping_indices} + +Ignores the skipping indexes specified if used by the query. + +Consider the following example: + +```sql +CREATE TABLE data +( + key Int, + x Int, + y Int, + INDEX x_idx x TYPE minmax GRANULARITY 1, + INDEX y_idx y TYPE minmax GRANULARITY 1, + INDEX xy_idx (x,y) TYPE minmax GRANULARITY 1 +) +Engine=MergeTree() +ORDER BY key; + +INSERT INTO data VALUES (1, 2, 3); + +SELECT * FROM data; +SELECT * FROM data SETTINGS ignore_data_skipping_indices=''; -- query will produce CANNOT_PARSE_TEXT error. +SELECT * FROM data SETTINGS ignore_data_skipping_indices='x_idx'; -- Ok. +SELECT * FROM data SETTINGS ignore_data_skipping_indices='na_idx'; -- Ok. + +SELECT * FROM data WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- query will produce INDEX_NOT_USED error, since xy_idx is explictly ignored. +SELECT * FROM data WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; +``` + +The query without ignoring any indexes: +```sql +EXPLAIN indexes = 1 SELECT * FROM data WHERE x = 1 AND y = 2; + +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 +``` + +Ignoring the `xy_idx` index: +```sql +EXPLAIN indexes = 1 SELECT * FROM data WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; + +Expression ((Projection + Before ORDER BY)) + Filter (WHERE) + ReadFromMergeTree (default.data) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 +``` + Works with tables in the MergeTree family. ## convert_query_to_cnf {#convert_query_to_cnf} From 689e0cabe0ca3cfc4b710a7426dbb1d363437984 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Tue, 30 May 2023 05:31:54 -0700 Subject: [PATCH 131/159] Add space to if --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ca6ab931499..c07d887588b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -979,7 +979,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd { auto index_helper = MergeTreeIndexFactory::instance().get(index); - if(!ignored_index_names.contains(index.name)) + if (!ignored_index_names.contains(index.name)) { if (index_helper->isMergeable()) { From 7c2b88a00eb1972fbd27b534ad578c2e69486605 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Thu, 1 Jun 2023 07:14:39 -0700 Subject: [PATCH 132/159] Make test invariant --- .../02771_ignore_data_skipping_indices.reference | 4 ---- .../0_stateless/02771_ignore_data_skipping_indices.sql | 6 +++--- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference index af1fce5ba13..786360783fd 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -1,8 +1,6 @@ 1 2 3 1 2 3 1 2 3 -Expression ((Projection + Before ORDER BY)) - Filter (WHERE) ReadFromMergeTree (default.data_02771) Indexes: PrimaryKey @@ -24,8 +22,6 @@ Expression ((Projection + Before ORDER BY)) Description: minmax GRANULARITY 1 Parts: 0/0 Granules: 0/0 -Expression ((Projection + Before ORDER BY)) - Filter (WHERE) ReadFromMergeTree (default.data_02771) Indexes: PrimaryKey diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index ab314ae5ea2..baa2d497863 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -20,7 +20,7 @@ SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='na_idx'; SELECT * FROM data_02771 WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- { serverError 277 } SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; -EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2; -EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; +SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; +SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; -DROP TABLE data_02771; +DROP TABLE data_02771; \ No newline at end of file From 1fa1215d1549e5887695cfd0f6bf4aaa61101fec Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Fri, 2 Jun 2023 07:21:22 -0700 Subject: [PATCH 133/159] Avoid UB --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../queries/0_stateless/02771_ignore_data_skipping_indices.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c07d887588b..4967de8424b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -954,7 +954,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (use_skip_indexes && settings.ignore_data_skipping_indices.changed) { const auto & indices = settings.ignore_data_skipping_indices.toString(); - Tokens tokens(indices.data(), &indices[indices.size()], settings.max_query_size); + Tokens tokens(indices.data(), indices.data() + indices.size(), settings.max_query_size); IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); Expected expected; diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index baa2d497863..289d5240b57 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -23,4 +23,4 @@ SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_ind SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; -DROP TABLE data_02771; \ No newline at end of file +DROP TABLE data_02771; From 45d000b71780cb7a022c6c0694d978596ac8fb96 Mon Sep 17 00:00:00 2001 From: Boris Kuschel Date: Mon, 5 Jun 2023 05:27:44 -0700 Subject: [PATCH 134/159] Turn off analyzer for test --- .../0_stateless/02771_ignore_data_skipping_indices.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index 289d5240b57..a49239e9de2 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -1,4 +1,8 @@ +SET allow_experimental_analyzer = 0; + DROP TABLE IF EXISTS data_02771; + + CREATE TABLE data_02771 ( key Int, From f3959aa9e16fd50ad5e7081c12a9a9948113e898 Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Wed, 7 Jun 2023 11:07:16 +0800 Subject: [PATCH 135/159] Update settings.md `max_final_threads` is now set to the number of cores by default. See https://github.com/ClickHouse/ClickHouse/pull/47915 --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5730503a670..8e2cd8d6027 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3155,7 +3155,7 @@ Possible values: - Positive integer. - 0 or 1 — Disabled. `SELECT` queries are executed in a single thread. -Default value: `16`. +Default value: the number of physical CPU cores. ## opentelemetry_start_trace_probability {#opentelemetry-start-trace-probability} From 4050b637f16554421423d92c501d9790deb42394 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 09:01:20 +0000 Subject: [PATCH 136/159] ALTER TABLE ADD INDEX: Add default GRANULARITY argument for secondary indexes - Related to #45451, which provides a default GRANULARITY when the skipping index is created in CREATE TABLE. --- docs/en/sql-reference/statements/alter/skipping-index.md | 2 +- src/Parsers/ParserCreateIndexQuery.cpp | 9 ++++++--- src/Parsers/ParserCreateQuery.cpp | 2 +- .../0_stateless/02534_default_granularity.reference | 1 + tests/queries/0_stateless/02534_default_granularity.sql | 7 ++++++- 5 files changed, 15 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/skipping-index.md b/docs/en/sql-reference/statements/alter/skipping-index.md index 67af76986da..4194731d33a 100644 --- a/docs/en/sql-reference/statements/alter/skipping-index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -10,7 +10,7 @@ sidebar_label: INDEX The following operations are available: -- `ALTER TABLE [db].table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type GRANULARITY value [FIRST|AFTER name]` - Adds index description to tables metadata. +- `ALTER TABLE [db].table_name [ON CLUSTER cluster] ADD INDEX name expression TYPE type [GRANULARITY value] [FIRST|AFTER name]` - Adds index description to tables metadata. - `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index ab31d3f9b7a..e878b347e62 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -36,17 +36,20 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected if (!data_type_p.parse(pos, type, expected)) return false; - if (!s_granularity.ignore(pos, expected)) - return false; + if (s_granularity.ignore(pos, expected)) + { + if (!granularity_p.parse(pos, granularity, expected)) + return false; + } if (!granularity_p.parse(pos, granularity, expected)) return false; auto index = std::make_shared(); index->part_of_create_index_query = true; - index->granularity = granularity->as().value.safeGet(); index->set(index->expr, expr); index->set(index->type, type); + index->granularity = granularity ? granularity->as().value.safeGet() : 1; node = index; return true; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 8cbfac91465..f975e8ba3c8 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -139,9 +139,9 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe auto index = std::make_shared(); index->name = name->as().name(); - index->granularity = granularity ? granularity->as().value.safeGet() : 1; index->set(index->expr, expr); index->set(index->type, type); + index->granularity = granularity ? granularity->as().value.safeGet() : 1; node = index; return true; diff --git a/tests/queries/0_stateless/02534_default_granularity.reference b/tests/queries/0_stateless/02534_default_granularity.reference index e60036653c9..0fe7fe0a1b3 100644 --- a/tests/queries/0_stateless/02534_default_granularity.reference +++ b/tests/queries/0_stateless/02534_default_granularity.reference @@ -1 +1,2 @@ CREATE TABLE default.users_02534\n(\n `id` Int16,\n `name` String,\n INDEX bf_idx name TYPE minmax GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.users_02534\n(\n `id` Int16,\n `name` String,\n INDEX bf_idx name TYPE minmax GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02534_default_granularity.sql b/tests/queries/0_stateless/02534_default_granularity.sql index 781df3ce934..e3de5fce7c8 100644 --- a/tests/queries/0_stateless/02534_default_granularity.sql +++ b/tests/queries/0_stateless/02534_default_granularity.sql @@ -1,4 +1,9 @@ DROP TABLE IF EXISTS users_02534; CREATE TABLE users_02534 (id Int16, name String, INDEX bf_idx(name) TYPE minmax) ENGINE=MergeTree ORDER BY id; SHOW CREATE TABLE users_02534; -DROP TABLE users_02534; \ No newline at end of file +DROP TABLE users_02534; + +CREATE TABLE users_02534 (id Int16, name String) ENGINE=MergeTree ORDER BY id; +ALTER TABLE users_02534 ADD INDEX bf_idx(name) TYPE minmax; +SHOW CREATE TABLE users_02534; +DROP TABLE users_02534; From c795eb03299f751dd4a0c69facb2d5a6bec101da Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 09:46:10 +0000 Subject: [PATCH 137/159] Temporarily disable a test --- tests/queries/0_stateless/02354_annoy_index.sql | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 170c048d420..0168fa04c6f 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -20,11 +20,12 @@ FROM tab ORDER BY L2Distance(embedding, [0.0, 0.0, 10.0]) LIMIT 3; -SELECT 'Reference ARRAYs with non-matching dimension are rejected'; -SELECT * -FROM tab -ORDER BY L2Distance(embedding, [0.0, 0.0]) -LIMIT 3; -- { serverError INCORRECT_QUERY } +-- Produces different error code with analyzer, TODO: check +-- SELECT 'Reference ARRAYs with non-matching dimension are rejected'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(embedding, [0.0, 0.0]) +-- LIMIT 3; -- { serverError INCORRECT_QUERY } SELECT 'WHERE type, L2Distance, check that index is used'; EXPLAIN indexes=1 From 7c800468349e4aea2c125459f44b581d14391a10 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 09:47:54 +0000 Subject: [PATCH 138/159] Revert "Remove clang-tidy exclude" This reverts commit 42c054789561920adf7ce4770968ba303a70f244. --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index ffed9e01df0..1a28f28f746 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -123,6 +123,7 @@ MergeTreeIndexAggregatorAnnoy::MergeTreeIndexAggregatorAnnoy( template MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy::getGranuleAndReset() { + // NOLINTNEXTLINE(*) index->build(static_cast(trees), /*number_of_threads=*/1); auto granule = std::make_shared>(index_name, index_sample_block, index); index = nullptr; From e0bc695e2d95085e1927b44fc2ad5a9d3384c1d3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Jun 2023 10:07:35 +0000 Subject: [PATCH 139/159] Use correct link format --- docs/en/sql-reference/aggregate-functions/index.md | 4 ++-- .../sql-reference/aggregate-functions/reference/argmax.md | 6 +++--- .../sql-reference/aggregate-functions/reference/argmin.md | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/index.md b/docs/en/sql-reference/aggregate-functions/index.md index ea270e83a3c..5d2229fbcce 100644 --- a/docs/en/sql-reference/aggregate-functions/index.md +++ b/docs/en/sql-reference/aggregate-functions/index.md @@ -4,7 +4,7 @@ sidebar_label: Aggregate Functions sidebar_position: 33 --- -# 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. @@ -73,7 +73,7 @@ FROM t_null_big └────────────────────┴─────────────────────┘ ``` -Also you can use [Tuple](../data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. +Also you can use [Tuple](/docs/en/sql-reference/data-types/tuple.md) to work around NULL skipping behavior. The a `Tuple` that contains only a `NULL` value is not `NULL`, so the aggregate functions won't skip that row because of that `NULL` value. ```sql SELECT diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 93e1fac6d67..8f10318838b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -5,8 +5,8 @@ sidebar_position: 106 # argMax -Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `max` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. +Calculates the `arg` value for a maximum `val` value. If there are several different values of `arg` for maximum values of `val`, returns the first of these values encountered. +Both parts the `arg` and the `max` behave as [aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md), they both [skip `Null`](/docs/en/sql-reference/aggregate-functions/index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -106,4 +106,4 @@ SELECT argMax(a, tuple(b)) FROM test; **See also** -- [Tuple](../../data-types/tuple.md) +- [Tuple](/docs/en/sql-reference/data-types/tuple.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 4e549e5b04c..47d4ab398de 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -6,7 +6,7 @@ sidebar_position: 105 # argMin Calculates the `arg` value for a minimum `val` value. If there are several different values of `arg` for minimum values of `val`, returns the first of these values encountered. -Both parts the `arg` and the `min` behave as [aggregate functions](../index.md), they both [skip `Null`](../index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. +Both parts the `arg` and the `min` behave as [aggregate functions](/docs/en/sql-reference/aggregate-functions/index.md), they both [skip `Null`](/docs/en/sql-reference/aggregate-functions/index.md#null-processing) during processing and return not `Null` values if not `Null` values are available. **Syntax** @@ -111,4 +111,4 @@ select argMin(a, tuple(b)) from test; **See also** -- [Tuple](../../data-types/tuple.md) +- [Tuple](/docs/en/sql-reference/data-types/tuple.md) From 35ef14482d785226a660c62fea558fdb91a1d26f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Wed, 7 Jun 2023 10:11:49 +0000 Subject: [PATCH 140/159] Fix keyword capitalization --- docs/en/sql-reference/aggregate-functions/reference/argmin.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmin.md b/docs/en/sql-reference/aggregate-functions/reference/argmin.md index 47d4ab398de..fdfce0833e0 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmin.md @@ -103,9 +103,9 @@ SELECT argMin((a, b), (b, a)), min(tuple(b, a)) FROM test; │ (NULL,NULL) │ (NULL,NULL) │ -- argMin returns (NULL,NULL) here because `Tuple` allows to don't skip `NULL` and min(tuple(b, a)) in this case is minimal value for this dataset └──────────────────────────────────┴──────────────────┘ -select argMin(a, tuple(b)) from test; +SELECT argMin(a, tuple(b)) FROM test; ┌─argMax(a, tuple(b))─┐ -│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b. +│ d │ -- `Tuple` can be used in `min` to not skip rows with `NULL` values as b. └─────────────────────┘ ``` From 036ddcd47baf88ab0c360efe647e01060d1ce636 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 7 Jun 2023 13:48:08 +0200 Subject: [PATCH 141/159] Fix excessive memory usage for FINAL (due to too much streams usage) (#50429) Previously it could create MergeTreeInOrder for each mark, however this could be very suboptimal, due to each MergeTreeInOrder has some memory overhead. Now, by collapsing all marks for one part together it is more memory effiecient. I've tried the query from the altinity wiki [1] and it decreases memory usage twice: SELECT * FROM repl_tbl FINAL WHERE key IN (SELECT toUInt32(number) FROM numbers(1000000) WHERE number % 50000 = 0) FORMAT Null - upstream: MemoryTracker: Peak memory usage (for query): 520.27 MiB. - patched: MemoryTracker: Peak memory usage (for query): 260.95 MiB. [1]: https://kb.altinity.com/engines/mergetree-table-engine-family/replacingmergetree/#multiple-keys And it could be not 2x and even more or less, it depends on the gaps in marks for reading (for example in my setup the memory usage increased a lot, from ~16GiB of RAM to >64GiB due to lots of marks and gaps). Signed-off-by: Azat Khuzhin --- src/Processors/QueryPlan/PartsSplitter.cpp | 35 ++++++++++----- ...inal_streams_data_skipping_index.reference | 43 +++++++++++++++++++ ...2780_final_streams_data_skipping_index.sql | 28 ++++++++++++ 3 files changed, 95 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference create mode 100644 tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 936182f8c00..9796e696f6c 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -126,7 +126,9 @@ std::pair, std::vector> split(RangesInDat return marks_in_current_layer < intersected_parts * 2; }; - result_layers.emplace_back(); + auto & current_layer = result_layers.emplace_back(); + /// Map part_idx into index inside layer, used to merge marks from the same part into one reader + std::unordered_map part_idx_in_layer; while (rows_in_current_layer < rows_per_layer || layers_intersection_is_too_big() || result_layers.size() == max_layers) { @@ -140,11 +142,16 @@ std::pair, std::vector> split(RangesInDat if (current.event == PartsRangesIterator::EventType::RangeEnd) { - result_layers.back().emplace_back( - parts[part_idx].data_part, - parts[part_idx].alter_conversions, - parts[part_idx].part_index_in_query, - MarkRanges{{current_part_range_begin[part_idx], current.range.end}}); + const auto & mark = MarkRange{current_part_range_begin[part_idx], current.range.end}; + auto it = part_idx_in_layer.emplace(std::make_pair(part_idx, current_layer.size())); + if (it.second) + current_layer.emplace_back( + parts[part_idx].data_part, + parts[part_idx].alter_conversions, + parts[part_idx].part_index_in_query, + MarkRanges{mark}); + else + current_layer[it.first->second].ranges.push_back(mark); current_part_range_begin.erase(part_idx); current_part_range_end.erase(part_idx); @@ -170,11 +177,17 @@ std::pair, std::vector> split(RangesInDat } for (const auto & [part_idx, last_mark] : current_part_range_end) { - result_layers.back().emplace_back( - parts[part_idx].data_part, - parts[part_idx].alter_conversions, - parts[part_idx].part_index_in_query, - MarkRanges{{current_part_range_begin[part_idx], last_mark + 1}}); + const auto & mark = MarkRange{current_part_range_begin[part_idx], last_mark + 1}; + auto it = part_idx_in_layer.emplace(std::make_pair(part_idx, current_layer.size())); + + if (it.second) + result_layers.back().emplace_back( + parts[part_idx].data_part, + parts[part_idx].alter_conversions, + parts[part_idx].part_index_in_query, + MarkRanges{mark}); + else + current_layer[it.first->second].ranges.push_back(mark); current_part_range_begin[part_idx] = current_part_range_end[part_idx]; } diff --git a/tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference new file mode 100644 index 00000000000..d7a540ae479 --- /dev/null +++ b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.reference @@ -0,0 +1,43 @@ +-- { echoOn } +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=1 +FORMAT LineAsString; +(Expression) +ExpressionTransform × 2 + (Filter) + FilterTransform × 2 + (ReadFromMergeTree) + ExpressionTransform × 2 + AggregatingSortedTransform 2 → 1 + ExpressionTransform × 2 + FilterSortedStreamByRange × 2 + Description: filter values in [(999424), +inf) + ExpressionTransform × 2 + MergeTreeInOrder × 2 0 → 1 + AggregatingSortedTransform + ExpressionTransform + FilterSortedStreamByRange + Description: filter values in [-inf, (999424)) + ExpressionTransform + MergeTreeInOrder 0 → 1 +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=0 +FORMAT LineAsString; +(Expression) +ExpressionTransform × 2 + (Filter) + FilterTransform × 2 + (ReadFromMergeTree) + ExpressionTransform × 2 + AggregatingSortedTransform 2 → 1 + ExpressionTransform × 2 + FilterSortedStreamByRange × 2 + Description: filter values in [(999424), +inf) + ExpressionTransform × 2 + MergeTreeInOrder × 2 0 → 1 + AggregatingSortedTransform + ExpressionTransform + FilterSortedStreamByRange + Description: filter values in [-inf, (999424)) + ExpressionTransform + MergeTreeInOrder 0 → 1 diff --git a/tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql new file mode 100644 index 00000000000..7de7a58e2e1 --- /dev/null +++ b/tests/queries/0_stateless/02780_final_streams_data_skipping_index.sql @@ -0,0 +1,28 @@ +-- Tags: no-random-merge-tree-settings, no-random-settings + +DROP TABLE IF EXISTS data; + +CREATE TABLE data +( + key Int, + v1 DateTime, + INDEX v1_index v1 TYPE minmax GRANULARITY 1 +) ENGINE=AggregatingMergeTree() +ORDER BY key +SETTINGS index_granularity=8192, min_bytes_for_wide_part=0, min_rows_for_wide_part=0; + +SYSTEM STOP MERGES data; + +-- generate 50% of marks that cannot be skipped with v1_index +-- this will create a gap in marks +INSERT INTO data SELECT number, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); +INSERT INTO data SELECT number+1e6, if(number/8192 % 2 == 0, now(), now() - INTERVAL 200 DAY) FROM numbers(1e6); + +-- { echoOn } +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=1 +FORMAT LineAsString; + +EXPLAIN PIPELINE SELECT * FROM data FINAL WHERE v1 >= now() - INTERVAL 180 DAY +SETTINGS max_threads=2, max_final_threads=2, force_data_skipping_indices='v1_index', use_skip_indexes_if_final=0 +FORMAT LineAsString; From 81cd3defd79331fc0af016c4c40a957be15a227a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 12:29:09 +0000 Subject: [PATCH 142/159] Fix expected results --- tests/queries/0_stateless/02354_annoy_index.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.reference b/tests/queries/0_stateless/02354_annoy_index.reference index 5bd1377d6f4..45515bc7733 100644 --- a/tests/queries/0_stateless/02354_annoy_index.reference +++ b/tests/queries/0_stateless/02354_annoy_index.reference @@ -9,7 +9,6 @@ ORDER BY type, L2Distance 1 [0,0,10] 5 [0,0,10.2] 4 [0,0,9.7] -Reference ARRAYs with non-matching dimension are rejected WHERE type, L2Distance, check that index is used Expression ((Projection + Before ORDER BY)) Limit (preliminary LIMIT (without OFFSET)) From bf6900f64ca7614a686dfaa56f87c84c43408506 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 7 Jun 2023 17:08:18 +0200 Subject: [PATCH 143/159] Write 1 part and do not use OPTIMIZE FINAL --- ...e_row_level_policy_lightweight_delete.sql.j2 | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 b/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 index 1e4258cef7e..0ec6b2ed144 100644 --- a/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 +++ b/tests/queries/0_stateless/02461_prewhere_row_level_policy_lightweight_delete.sql.j2 @@ -8,25 +8,10 @@ ORDER BY (SiteId, DateVisit) SETTINGS index_granularity = {{ index_granularity }}, min_bytes_for_wide_part = 0; -- Insert some data to have 110K rows in the range 2022-08-10 .. 2022-08-20 and some more rows before and after that range -insert into url_na_log select 209, '2022-08-09' from numbers(10000); -insert into url_na_log select 209, '2022-08-10' from numbers(10000); -insert into url_na_log select 209, '2022-08-11' from numbers(10000); -insert into url_na_log select 209, '2022-08-12' from numbers(10000); -insert into url_na_log select 209, '2022-08-13' from numbers(10000); -insert into url_na_log select 209, '2022-08-14' from numbers(10000); -insert into url_na_log select 209, '2022-08-15' from numbers(10000); -insert into url_na_log select 209, '2022-08-16' from numbers(10000); -insert into url_na_log select 209, '2022-08-17' from numbers(10000); -insert into url_na_log select 209, '2022-08-18' from numbers(10000); -insert into url_na_log select 209, '2022-08-19' from numbers(10000); -insert into url_na_log select 209, '2022-08-20' from numbers(10000); -insert into url_na_log select 209, '2022-08-21' from numbers(10000); - +insert into url_na_log select 209, ('2022-08-09'::Date + INTERVAL intDiv(number,10000) DAY) from numbers(130000) SETTINGS max_insert_block_size=200000; SET mutations_sync=2; -OPTIMIZE TABLE url_na_log FINAL; - -- { echoOn } SELECT count() FROM url_na_log; From c12c62a7fb7072772f423c907c467c39762ec5a7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Jun 2023 18:22:53 +0200 Subject: [PATCH 144/159] Update src/Parsers/ParserCreateIndexQuery.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Parsers/ParserCreateIndexQuery.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index e878b347e62..7323c5da141 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -42,9 +42,6 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected return false; } - if (!granularity_p.parse(pos, granularity, expected)) - return false; - auto index = std::make_shared(); index->part_of_create_index_query = true; index->set(index->expr, expr); From e1ceb01ad0b8988ec70818472740fdeff5acf112 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 7 Jun 2023 19:45:27 +0200 Subject: [PATCH 145/159] Parallel replicas for short queries (#50639) --- src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 39 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 7 ++++ tests/broken_tests.txt | 2 +- ...lel_replicas_automatic_disabling.reference | 2 + ..._parallel_replicas_automatic_disabling.sql | 15 +++++++ 6 files changed, 65 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference create mode 100644 tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0037acedede..a87e321bed2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -160,6 +160,7 @@ class IColumn; M(UInt64, allow_experimental_parallel_reading_from_replicas, 0, "Use all the replicas from a shard for SELECT query execution. Reading is parallelized and coordinated dynamically. 0 - disabled, 1 - enabled, silently disable them in case of failure, 2 - enabled, throw an exception in case of failure", 0) \ M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \ M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ + M(UInt64, parallel_replicas_min_number_of_granules_to_enable, 0, "If the number of marks to read is less than the value of this setting - parallel replicas will be disabled", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e806e1bb93f..2f6870f8b41 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7154,6 +7154,9 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( /// Parallel replicas if (query_context->canUseParallelReplicasOnInitiator() && to_stage >= QueryProcessingStage::WithMergeableState) { + if (!canUseParallelReplicasBasedOnPKAnalysis(query_context, storage_snapshot, query_info)) + return QueryProcessingStage::Enum::FetchColumns; + /// ReplicatedMergeTree if (supportsReplication()) return QueryProcessingStage::Enum::WithMergeableState; @@ -7179,6 +7182,42 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( } +bool MergeTreeData::canUseParallelReplicasBasedOnPKAnalysis( + ContextPtr query_context, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info) const +{ + const auto & snapshot_data = assert_cast(*storage_snapshot->data); + const auto & parts = snapshot_data.parts; + + MergeTreeDataSelectExecutor reader(*this); + auto result_ptr = reader.estimateNumMarksToRead( + parts, + query_info.prewhere_info, + storage_snapshot->getMetadataForQuery()->getColumns().getAll().getNames(), + storage_snapshot->metadata, + storage_snapshot->metadata, + query_info, + /*added_filter_nodes*/ActionDAGNodes{}, + query_context, + query_context->getSettingsRef().max_threads); + + if (result_ptr->error()) + std::rethrow_exception(std::get(result_ptr->result)); + + LOG_TRACE(log, "Estimated number of granules to read is {}", result_ptr->marks()); + + bool decision = result_ptr->marks() >= query_context->getSettingsRef().parallel_replicas_min_number_of_granules_to_enable; + + if (!decision) + LOG_DEBUG(log, "Parallel replicas will be disabled, because the estimated number of granules to read {} is less than the threshold which is {}", + result_ptr->marks(), + query_context->getSettingsRef().parallel_replicas_min_number_of_granules_to_enable); + + return decision; +} + + MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const { MergeTreeData * src_data = dynamic_cast(&source_table); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2f254f9a787..b1e1e43bd0b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1536,6 +1536,13 @@ private: static MutableDataPartPtr asMutableDeletingPart(const DataPartPtr & part); mutable TemporaryParts temporary_parts; + + /// Estimate the number of marks to read to make a decision whether to enable parallel replicas (distributed processing) or not + /// Note: it could be very rough. + bool canUseParallelReplicasBasedOnPKAnalysis( + ContextPtr query_context, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info) const; }; /// RAII struct to record big parts that are submerging or emerging. diff --git a/tests/broken_tests.txt b/tests/broken_tests.txt index 02935712325..7ee497973b8 100644 --- a/tests/broken_tests.txt +++ b/tests/broken_tests.txt @@ -137,4 +137,4 @@ 02534_s3_cluster_insert_select_schema_inference 02764_parallel_replicas_plain_merge_tree 02765_parallel_replicas_final_modifier - +02784_parallel_replicas_automatic_disabling diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference new file mode 100644 index 00000000000..af81158ecae --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.reference @@ -0,0 +1,2 @@ +10 +1 diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql new file mode 100644 index 00000000000..b2f674ddb64 --- /dev/null +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_disabling.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_parallel_replicas_automatic_disabling; +CREATE TABLE test_parallel_replicas_automatic_disabling (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_parallel_replicas_automatic_disabling SELECT * FROM numbers(10); + +SYSTEM FLUSH LOGS; + +SET skip_unavailable_shards=1, allow_experimental_parallel_reading_from_replicas=1, max_parallel_replicas=3, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, parallel_replicas_min_number_of_granules_to_enable=10000; +SET send_logs_level='error'; +SELECT count() FROM test_parallel_replicas_automatic_disabling WHERE NOT ignore(*); + +SYSTEM FLUSH LOGS; + +SELECT count() > 0 FROM system.text_log WHERE event_time >= now() - INTERVAL 2 MINUTE AND message LIKE '%Parallel replicas will be disabled, because the estimated number of granules to read%'; + +DROP TABLE test_parallel_replicas_automatic_disabling; From c0f2141bd0432c7ebcab5ee0ef033141194fd59d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 7 Jun 2023 19:51:41 +0200 Subject: [PATCH 146/159] Revert "date_trunc function to always return DateTime type" (#50670) --- src/Functions/date_trunc.cpp | 98 ++++++++----------- .../00189_time_zones_long.reference | 56 +++++------ ...21_datetime64_compatibility_long.reference | 8 +- 3 files changed, 71 insertions(+), 91 deletions(-) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 87fff0b7f3c..016b8f4da5e 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -1,6 +1,6 @@ #include -#include #include +#include #include #include #include @@ -25,7 +25,7 @@ class FunctionDateTrunc : public IFunction public: static constexpr auto name = "dateTrunc"; - explicit FunctionDateTrunc(ContextPtr context_) : context(context_) { } + explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {} static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } @@ -39,58 +39,51 @@ public: { /// The first argument is a constant string with the name of datepart. - intermediate_type_is_date = false; + auto result_type_is_date = false; String datepart_param; - auto check_first_argument = [&] - { + auto check_first_argument = [&] { const ColumnConst * datepart_column = checkAndGetColumnConst(arguments[0].column.get()); if (!datepart_column) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument for function {} must be constant string: " - "name of datepart", - getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: " + "name of datepart", getName()); datepart_param = datepart_column->getValue(); if (datepart_param.empty()) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty", + getName()); if (!IntervalKind::tryParseString(datepart_param, datepart_kind)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName()); - intermediate_type_is_date = (datepart_kind == IntervalKind::Year) || (datepart_kind == IntervalKind::Quarter) - || (datepart_kind == IntervalKind::Month) || (datepart_kind == IntervalKind::Week); + result_type_is_date = (datepart_kind == IntervalKind::Year) + || (datepart_kind == IntervalKind::Quarter) || (datepart_kind == IntervalKind::Month) + || (datepart_kind == IntervalKind::Week); }; bool second_argument_is_date = false; - auto check_second_argument = [&] - { + auto check_second_argument = [&] { if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 2nd argument of function {}. " - "Should be a date or a date with time", - arguments[1].type->getName(), - getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " + "Should be a date or a date with time", arguments[1].type->getName(), getName()); second_argument_is_date = isDate(arguments[1].type); - if (second_argument_is_date - && ((datepart_kind == IntervalKind::Hour) || (datepart_kind == IntervalKind::Minute) - || (datepart_kind == IntervalKind::Second))) + if (second_argument_is_date && ((datepart_kind == IntervalKind::Hour) + || (datepart_kind == IntervalKind::Minute) || (datepart_kind == IntervalKind::Second))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName()); }; - auto check_timezone_argument = [&] - { + auto check_timezone_argument = [&] { if (!WhichDataType(arguments[2].type).isString()) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument of function {}. " + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. " "This argument is optional and must be a constant string with timezone name", - arguments[2].type->getName(), - getName()); + arguments[2].type->getName(), getName()); + + if (second_argument_is_date && result_type_is_date) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "The timezone argument of function {} with datepart '{}' " + "is allowed only when the 2nd argument has the type DateTime", + getName(), datepart_param); }; if (arguments.size() == 2) @@ -106,14 +99,15 @@ public: } else { - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", - getName(), - arguments.size()); + getName(), arguments.size()); } - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1)); + if (result_type_is_date) + return std::make_shared(); + else + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1)); } bool useDefaultImplementationForConstants() const override { return true; } @@ -130,40 +124,26 @@ public: auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context); - ColumnPtr truncated_column; - auto date_type = std::make_shared(); - if (arguments.size() == 2) - truncated_column = to_start_of_interval->build(temp_columns) - ->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count); - else - { - temp_columns[2] = arguments[2]; - truncated_column = to_start_of_interval->build(temp_columns) - ->execute(temp_columns, intermediate_type_is_date ? date_type : result_type, input_rows_count); - } + return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count); - if (!intermediate_type_is_date) - return truncated_column; - - ColumnsWithTypeAndName temp_truncated_column(1); - temp_truncated_column[0] = {truncated_column, date_type, ""}; - - auto to_date_time_or_default = FunctionFactory::instance().get("toDateTime", context); - return to_date_time_or_default->build(temp_truncated_column)->execute(temp_truncated_column, result_type, input_rows_count); + temp_columns[2] = arguments[2]; + return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count); } - bool hasInformationAboutMonotonicity() const override { return true; } + bool hasInformationAboutMonotonicity() const override + { + return true; + } Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { - return {.is_monotonic = true, .is_always_monotonic = true}; + return { .is_monotonic = true, .is_always_monotonic = true }; } private: ContextPtr context; mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second; - mutable bool intermediate_type_is_date = false; }; } diff --git a/tests/queries/0_stateless/00189_time_zones_long.reference b/tests/queries/0_stateless/00189_time_zones_long.reference index a4287217a19..7d2ad3c8a01 100644 --- a/tests/queries/0_stateless/00189_time_zones_long.reference +++ b/tests/queries/0_stateless/00189_time_zones_long.reference @@ -258,18 +258,18 @@ toUnixTimestamp 1426415400 1426415400 date_trunc -2019-01-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-10-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 +2019-01-01 +2020-01-01 +2020-01-01 +2019-10-01 +2020-01-01 +2020-01-01 +2019-12-01 +2020-01-01 +2020-01-01 +2019-12-30 +2019-12-30 +2019-12-30 2019-12-31 00:00:00 2020-01-01 00:00:00 2020-01-02 00:00:00 @@ -282,18 +282,18 @@ date_trunc 2019-12-31 20:11:22 2020-01-01 12:11:22 2020-01-02 05:11:22 -2019-01-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-10-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 -2019-12-30 00:00:00 +2019-01-01 +2020-01-01 +2020-01-01 +2019-10-01 +2020-01-01 +2020-01-01 +2019-12-01 +2020-01-01 +2020-01-01 +2019-12-30 +2019-12-30 +2019-12-30 2019-12-31 00:00:00 2020-01-01 00:00:00 2020-01-02 00:00:00 @@ -306,8 +306,8 @@ date_trunc 2019-12-31 20:11:22 2020-01-01 12:11:22 2020-01-02 05:11:22 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2020-01-01 00:00:00 -2019-12-30 00:00:00 +2020-01-01 +2020-01-01 +2020-01-01 +2019-12-30 2020-01-01 00:00:00 diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference index 2f56230db37..a946a114bf4 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility_long.reference @@ -135,13 +135,13 @@ Code: 43 ------------------------------------------ SELECT date_trunc(\'year\', N, \'Asia/Istanbul\') Code: 43 -"DateTime('Asia/Istanbul')","2019-01-01 00:00:00" -"DateTime('Asia/Istanbul')","2019-01-01 00:00:00" +"Date","2019-01-01" +"Date","2019-01-01" ------------------------------------------ SELECT date_trunc(\'month\', N, \'Asia/Istanbul\') Code: 43 -"DateTime('Asia/Istanbul')","2019-09-01 00:00:00" -"DateTime('Asia/Istanbul')","2019-09-01 00:00:00" +"Date","2019-09-01" +"Date","2019-09-01" ------------------------------------------ SELECT date_trunc(\'day\', N, \'Asia/Istanbul\') "DateTime('Asia/Istanbul')","2019-09-16 00:00:00" From f3b5a87a66baa6ffed9cba8caa52e4c4a63cfc3d Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 6 Jun 2023 19:03:23 +0000 Subject: [PATCH 147/159] Fixing crash in anti/semi join --- src/Interpreters/HashJoin.cpp | 12 ++-- src/Interpreters/TableJoin.h | 15 ++++- src/Planner/PlannerJoinTree.cpp | 7 ++- .../02771_semi_join_use_nulls.reference | 0 .../02771_semi_join_use_nulls.sql.j2 | 60 +++++++++++++++++++ 5 files changed, 85 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02771_semi_join_use_nulls.reference create mode 100644 tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9306c9b99eb..191dd74928d 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -679,7 +679,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || - isRightOrFull(kind) || + isRightOrFull(kind) || strictness == JoinStrictness::Semi || strictness == JoinStrictness::Anti || multiple_disjuncts; if (save_key_columns) { @@ -707,8 +707,9 @@ Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block for (const auto & sample_column : saved_block_sample_.getColumnsWithTypeAndName()) { ColumnWithTypeAndName column = block.getByName(sample_column.name); - if (sample_column.column->isNullable()) - JoinCommon::convertColumnToNullable(column); + + /// There's no optimization for right side const columns. Remove constness if any. + column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); if (column.column->lowCardinality() && !sample_column.column->lowCardinality()) { @@ -716,8 +717,9 @@ Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block column.type = removeLowCardinality(column.type); } - /// There's no optimization for right side const columns. Remove constness if any. - column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst()); + if (sample_column.column->isNullable()) + JoinCommon::convertColumnToNullable(column); + structured_block.insert(std::move(column)); } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 0e0c905e30c..4a020684793 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -232,8 +232,19 @@ public: bool allowParallelHashJoin() const; bool joinUseNulls() const { return join_use_nulls; } - bool forceNullableRight() const { return join_use_nulls && isLeftOrFull(kind()); } - bool forceNullableLeft() const { return join_use_nulls && isRightOrFull(kind()); } + + /// Join use nulls doen't make sense for semi and anti joins + /// Only columns from corresponding table should be used, values in other table are undefined. + bool forceNullableRight() const + { + return join_use_nulls && isLeftOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + } + + bool forceNullableLeft() const + { + return join_use_nulls && isRightOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + } + size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 9672738ae6b..c0c6d301d88 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -873,10 +873,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ JoinClausesAndActions join_clauses_and_actions; JoinKind join_kind = join_node.getKind(); + JoinStrictness join_strictness = join_node.getStrictness(); std::optional join_constant; - if (join_node.getStrictness() == JoinStrictness::All) + if (join_strictness == JoinStrictness::All) join_constant = tryExtractConstantFromJoinNode(join_table_expression); if (join_constant) @@ -996,7 +997,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ plan_to_add_cast.addStep(std::move(cast_join_columns_step)); }; - if (join_use_nulls) + /// Join use nulls doen't make sense for semi and anti joins + /// Only columns from corresponding table should be used, values in other table are undefined. + if (join_use_nulls && join_strictness != JoinStrictness::Semi && join_strictness != JoinStrictness::Anti) { if (isFull(join_kind)) { diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.reference b/tests/queries/0_stateless/02771_semi_join_use_nulls.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 new file mode 100644 index 00000000000..308df9a6094 --- /dev/null +++ b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 @@ -0,0 +1,60 @@ + +{% for allow_experimental_analyzer in [0, 1] -%} +{% for join_use_nulls in [0, 1] -%} +{% for kind in ['LEFT', 'RIGHT'] -%} +{% for strictness in ['SEMI', 'ANTI'] -%} +{% for maybe_materialize in ['', 'materialize'] -%} + +SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; + +SET join_use_nulls = {{ join_use_nulls }}; + +-- FORMAT Null is used because one side is undefined (e.g. for SEMI LEFT only columns from the left side contain values) + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(1)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(2)) AS id) AS d +USING (id) +FORMAT Null +; + +SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d +ON a.id = d.id +FORMAT Null +; + +SELECT id > 1, d.idd FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a +{{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS idd) AS d +ON a.id = d.idd +FORMAT Null +; + +{% endfor -%} +{% endfor -%} +{% endfor -%} +{% endfor -%} +{% endfor -%} From a268250aff8fad35828641d6242c9d6d0e9ee3a3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Jun 2023 13:25:06 +0000 Subject: [PATCH 148/159] Fixes for LowCardinality Nullable in HashJoin --- src/Columns/ColumnLowCardinality.cpp | 5 + src/Columns/tests/gtest_low_cardinality.cpp | 13 ++ src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/JoinUtils.cpp | 18 +-- src/Interpreters/TableJoin.h | 6 +- src/Planner/PlannerJoinTree.cpp | 4 +- .../02771_semi_join_use_nulls.reference | 112 ++++++++++++++++++ .../02771_semi_join_use_nulls.sql.j2 | 9 -- 8 files changed, 140 insertions(+), 29 deletions(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 4f9ab8215be..9269ea4ee4d 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -313,6 +313,11 @@ MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const MutableColumnPtr ColumnLowCardinality::cloneNullable() const { auto res = cloneFinalized(); + /* Compact required not to share dictionary. + * If `shared` flag is not set `cloneFinalized` will return shallow copy + * and `nestedToNullable` will mutate source column. + */ + assert_cast(*res).compactInplace(); assert_cast(*res).nestedToNullable(); return res; } diff --git a/src/Columns/tests/gtest_low_cardinality.cpp b/src/Columns/tests/gtest_low_cardinality.cpp index 3ffc88f6a7d..5e01279b7df 100644 --- a/src/Columns/tests/gtest_low_cardinality.cpp +++ b/src/Columns/tests/gtest_low_cardinality.cpp @@ -48,3 +48,16 @@ TEST(ColumnLowCardinality, Insert) testLowCardinalityNumberInsert(std::make_shared()); testLowCardinalityNumberInsert(std::make_shared()); } + +TEST(ColumnLowCardinality, Clone) +{ + auto data_type = std::make_shared(); + auto low_cardinality_type = std::make_shared(data_type); + auto column = low_cardinality_type->createColumn(); + ASSERT_FALSE(assert_cast(*column).nestedIsNullable()); + + auto nullable_column = assert_cast(*column).cloneNullable(); + + ASSERT_TRUE(assert_cast(*nullable_column).nestedIsNullable()); + ASSERT_FALSE(assert_cast(*column).nestedIsNullable()); +} diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 191dd74928d..c4d1615a119 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -679,7 +679,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) || table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) || - isRightOrFull(kind) || strictness == JoinStrictness::Semi || strictness == JoinStrictness::Anti || + isRightOrFull(kind) || multiple_disjuncts; if (save_key_columns) { diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index a05b58e14a1..67ee2a64264 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -160,16 +160,14 @@ static ColumnPtr tryConvertColumnToNullable(ColumnPtr col) if (col->lowCardinality()) { - auto mut_col = IColumn::mutate(std::move(col)); - ColumnLowCardinality * col_lc = assert_cast(mut_col.get()); - if (col_lc->nestedIsNullable()) + const ColumnLowCardinality & col_lc = assert_cast(*col); + if (col_lc.nestedIsNullable()) { - return mut_col; + return col; } - else if (col_lc->nestedCanBeInsideNullable()) + else if (col_lc.nestedCanBeInsideNullable()) { - col_lc->nestedToNullable(); - return mut_col; + return col_lc.cloneNullable(); } } else if (const ColumnConst * col_const = checkAndGetColumn(*col)) @@ -232,11 +230,7 @@ void removeColumnNullability(ColumnWithTypeAndName & column) if (column.column && column.column->lowCardinality()) { - auto mut_col = IColumn::mutate(std::move(column.column)); - ColumnLowCardinality * col_as_lc = typeid_cast(mut_col.get()); - if (col_as_lc && col_as_lc->nestedIsNullable()) - col_as_lc->nestedRemoveNullable(); - column.column = std::move(mut_col); + column.column = assert_cast(column.column.get())->cloneWithDefaultOnNull(); } } else diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4a020684793..6737cd8f13a 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -233,16 +233,14 @@ public: bool joinUseNulls() const { return join_use_nulls; } - /// Join use nulls doen't make sense for semi and anti joins - /// Only columns from corresponding table should be used, values in other table are undefined. bool forceNullableRight() const { - return join_use_nulls && isLeftOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + return join_use_nulls && isLeftOrFull(kind()); } bool forceNullableLeft() const { - return join_use_nulls && isRightOrFull(kind()) && strictness() != JoinStrictness::Semi && strictness() != JoinStrictness::Anti; + return join_use_nulls && isRightOrFull(kind()); } size_t defaultMaxBytes() const { return default_max_bytes; } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c0c6d301d88..d875cc38bce 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -997,9 +997,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ plan_to_add_cast.addStep(std::move(cast_join_columns_step)); }; - /// Join use nulls doen't make sense for semi and anti joins - /// Only columns from corresponding table should be used, values in other table are undefined. - if (join_use_nulls && join_strictness != JoinStrictness::Semi && join_strictness != JoinStrictness::Anti) + if (join_use_nulls) { if (isFull(join_kind)) { diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.reference b/tests/queries/0_stateless/02771_semi_join_use_nulls.reference index e69de29bb2d..8d4b1a3a75e 100644 --- a/tests/queries/0_stateless/02771_semi_join_use_nulls.reference +++ b/tests/queries/0_stateless/02771_semi_join_use_nulls.reference @@ -0,0 +1,112 @@ +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 1 +0 1 +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 0 +0 0 +0 \N +0 0 +0 0 +0 0 +0 2 +0 2 +0 \N +0 0 +0 \N +0 0 +0 \N +0 \N +0 \N +0 0 +0 \N +0 0 +0 \N +0 \N +0 1 +0 1 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +\N 2 +\N 2 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 1 +0 1 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 2 +1 2 +0 \N +0 0 +0 0 +0 0 +0 \N +0 \N +0 \N +0 0 +0 0 +0 0 +0 \N +0 \N +0 1 +0 1 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 2 +1 2 diff --git a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 index 308df9a6094..37b2e63761b 100644 --- a/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 +++ b/tests/queries/0_stateless/02771_semi_join_use_nulls.sql.j2 @@ -9,48 +9,39 @@ SET allow_experimental_analyzer = {{ allow_experimental_analyzer }}; SET join_use_nulls = {{ join_use_nulls }}; --- FORMAT Null is used because one side is undefined (e.g. for SEMI LEFT only columns from the left side contain values) - SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(toNullable(0))) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(1)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(2)) AS id) AS d USING (id) -FORMAT Null ; SELECT id > 1, d.id FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id) AS d ON a.id = d.id -FORMAT Null ; SELECT id > 1, d.idd FROM (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS id, 1 AS value) AS a {{ strictness }} {{ kind }} JOIN (SELECT {{ maybe_materialize }}(toLowCardinality(0)) AS idd) AS d ON a.id = d.idd -FORMAT Null ; {% endfor -%} From b11f744252b486ca0ba25deeb07181b4025e0edf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Jun 2023 20:33:08 +0200 Subject: [PATCH 149/159] Correctly disable async insert with deduplication when it's not needed (#50663) * Correctly disable async insert when it's not used * Better * Add comment * Better * Fix tests --------- Co-authored-by: Nikita Mikhaylov --- src/Core/ExternalTable.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 4 +-- .../Transforms/CreatingSetsTransform.cpp | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 9 +++--- .../Transforms/buildPushingToViewsChain.h | 2 ++ src/Server/GRPCServer.cpp | 2 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/HDFS/StorageHDFS.h | 2 +- src/Storages/Hive/StorageHive.cpp | 2 +- src/Storages/Hive/StorageHive.h | 2 +- src/Storages/IStorage.h | 5 +++- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/Kafka/StorageKafka.h | 3 +- .../MeiliSearch/StorageMeiliSearch.cpp | 2 +- src/Storages/MeiliSearch/StorageMeiliSearch.h | 2 +- src/Storages/NATS/StorageNATS.cpp | 2 +- src/Storages/NATS/StorageNATS.h | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageDistributed.cpp | 4 +-- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.cpp | 3 +- src/Storages/StorageFile.h | 3 +- src/Storages/StorageJoin.cpp | 4 +-- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageKeeperMap.cpp | 2 +- src/Storages/StorageKeeperMap.h | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageLog.h | 2 +- src/Storages/StorageMaterializedMySQL.h | 2 +- src/Storages/StorageMaterializedView.cpp | 4 +-- src/Storages/StorageMaterializedView.h | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMongoDB.h | 3 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageMySQL.h | 2 +- src/Storages/StorageNull.h | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StoragePostgreSQL.h | 2 +- src/Storages/StorageProxy.h | 4 +-- src/Storages/StorageReplicatedMergeTree.cpp | 4 +-- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSQLite.h | 2 +- src/Storages/StorageSet.cpp | 2 +- src/Storages/StorageSet.h | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTableFunction.h | 5 ++-- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageURL.h | 2 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/StorageXDBC.h | 2 +- .../System/StorageSystemZooKeeper.cpp | 2 +- src/Storages/System/StorageSystemZooKeeper.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- src/Storages/tests/gtest_storage_log.cpp | 2 +- ...sable_async_with_dedup_correctly.reference | 17 +++++++++++ ...2784_disable_async_with_dedup_correctly.sh | 29 +++++++++++++++++++ 71 files changed, 136 insertions(+), 78 deletions(-) create mode 100644 tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference create mode 100755 tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 0f880ed967f..676af280cad 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -167,7 +167,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, auto temporary_table = TemporaryTableHolder(getContext(), ColumnsDescription{columns}, {}); auto storage = temporary_table.getTable(); getContext()->addExternalTable(data->table_name, std::move(temporary_table)); - auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext()); + auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false); /// Write data auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*data->pipe)); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 08862032007..2901f2e23d0 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -170,7 +170,7 @@ public: else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) { auto external_table = external_storage_holder->getTable(); - auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); + auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false); auto io = interpreter->execute(); io.pipeline.complete(std::move(table_out)); CompletedPipelineExecutor executor(io.pipeline); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e87b16f0e9d..078499fb013 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -282,7 +282,7 @@ Chain InterpreterInsertQuery::buildSink( /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. if (table->noPushingToViews() && !no_destination) { - auto sink = table->write(query_ptr, metadata_snapshot, context_ptr); + auto sink = table->write(query_ptr, metadata_snapshot, context_ptr, async_insert); sink->setRuntimeData(thread_status, elapsed_counter_ms); out.addSource(std::move(sink)); } @@ -290,7 +290,7 @@ Chain InterpreterInsertQuery::buildSink( { out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, - thread_status_holder, running_group, elapsed_counter_ms); + thread_status_holder, running_group, elapsed_counter_ms, async_insert); } return out; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e3ae2d4fd4e..c6ac8bea5ba 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -91,7 +91,7 @@ void CreatingSetsTransform::startSubquery() if (subquery.table) /// TODO: make via port - table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); + table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false)); done_with_set = !subquery.set_in_progress; done_with_table = !subquery.table; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 31bab46b868..0d33f250b5d 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -196,6 +196,7 @@ Chain buildPushingToViewsChain( ThreadStatusesHolderPtr thread_status_holder, ThreadGroupPtr running_group, std::atomic_uint64_t * elapsed_counter_ms, + bool async_insert, const Block & live_view_header) { checkStackSize(); @@ -347,7 +348,7 @@ Chain buildPushingToViewsChain( out = buildPushingToViewsChain( view, view_metadata_snapshot, insert_context, ASTPtr(), /* no_destination= */ true, - thread_status_holder, running_group, view_counter_ms, storage_header); + thread_status_holder, running_group, view_counter_ms, async_insert, storage_header); } else if (auto * window_view = dynamic_cast(view.get())) { @@ -356,13 +357,13 @@ Chain buildPushingToViewsChain( out = buildPushingToViewsChain( view, view_metadata_snapshot, insert_context, ASTPtr(), /* no_destination= */ true, - thread_status_holder, running_group, view_counter_ms); + thread_status_holder, running_group, view_counter_ms, async_insert); } else out = buildPushingToViewsChain( view, view_metadata_snapshot, insert_context, ASTPtr(), /* no_destination= */ false, - thread_status_holder, running_group, view_counter_ms); + thread_status_holder, running_group, view_counter_ms, async_insert); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), @@ -444,7 +445,7 @@ Chain buildPushingToViewsChain( /// Do not push to destination table if the flag is set else if (!no_destination) { - auto sink = storage->write(query_ptr, metadata_snapshot, context); + auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 0f413bee5c6..53aceeda1cc 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -69,6 +69,8 @@ Chain buildPushingToViewsChain( ThreadGroupPtr running_group, /// Counter to measure time spent separately per view. Should be improved. std::atomic_uint64_t * elapsed_counter_ms, + /// True if it's part of async insert flush + bool async_insert, /// LiveView executes query itself, it needs source block structure. const Block & live_view_header = {}); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 7b8eaa21947..e335d247a82 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1101,7 +1101,7 @@ namespace { /// The data will be written directly to the table. auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto sink = storage->write(ASTPtr(), metadata_snapshot, query_context); + auto sink = storage->write(ASTPtr(), metadata_snapshot, query_context, /*async_insert=*/false); std::unique_ptr buf = std::make_unique(external_table.data().data(), external_table.data().size()); buf = wrapReadBufferWithCompressionMethod(std::move(buf), chooseCompressionMethod("", external_table.compression_type())); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 0522b6d8a48..1ded7d97248 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1692,7 +1692,7 @@ bool TCPHandler::receiveData(bool scalar) } auto metadata_snapshot = storage->getInMemoryMetadataPtr(); /// The data will be written directly to the table. - QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context)); + QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context, /*async_insert=*/false)); PushingPipelineExecutor executor(temporary_table_out); executor.start(); executor.push(block); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 98c2579e355..194a8f982d8 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -624,7 +624,7 @@ Pipe StorageHDFS::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_) +SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/) { String current_uri = uris.back(); diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index b123834e981..c6226c2618d 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -41,7 +41,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; void truncate( const ASTPtr & query, diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index f554a14ec75..00c942fd56b 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -905,7 +905,7 @@ HiveFiles StorageHive::collectHiveFiles( return hive_files; } -SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot*/, ContextPtr /*context*/) +SinkToStoragePtr StorageHive::write(const ASTPtr & /*query*/, const StorageMetadataPtr & /* metadata_snapshot*/, ContextPtr /*context*/, bool /*async_insert*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not implemented for StorageHive"); } diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index b4afb2421b1..604df70f4d0 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -61,7 +61,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) override; + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool async_insert) override; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index c163e8107ac..d44772850fd 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -402,11 +402,14 @@ public: * passed in all parts of the returned streams. Storage metadata can be * changed during lifetime of the returned streams, but the snapshot is * guaranteed to be immutable. + * + * async_insert - set to true if the write is part of async insert flushing */ virtual SinkToStoragePtr write( const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr /*context*/) + ContextPtr /*context*/, + bool /*async_insert*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not supported by storage {}", getName()); } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7d504833a0a..2aba76c1a3f 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -374,7 +374,7 @@ Pipe StorageKafka::read( } -SinkToStoragePtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageKafka::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto modified_context = Context::createCopy(local_context); modified_context->applySettingsChanges(settings_adjustments); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 3559129cf74..09aa091ef18 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -60,7 +60,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; /// We want to control the number of rows in a chunk inserted into Kafka bool prefersLargeBlocks() const override { return false; } diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp index e7350d38a20..5d77fc080a4 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.cpp +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.cpp @@ -137,7 +137,7 @@ Pipe StorageMeiliSearch::read( return Pipe(std::make_shared(config, sample_block, max_block_size, route, kv_pairs_params)); } -SinkToStoragePtr StorageMeiliSearch::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageMeiliSearch::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { LOG_TRACE(log, "Trying update index: {}", config.index); return std::make_shared(config, metadata_snapshot->getSampleBlock(), local_context); diff --git a/src/Storages/MeiliSearch/StorageMeiliSearch.h b/src/Storages/MeiliSearch/StorageMeiliSearch.h index 41c1db53437..77cd2afb80a 100644 --- a/src/Storages/MeiliSearch/StorageMeiliSearch.h +++ b/src/Storages/MeiliSearch/StorageMeiliSearch.h @@ -26,7 +26,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) override; static MeiliSearchConfiguration getConfiguration(ASTs engine_args, ContextPtr context); diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index aa4ec77b0d8..a3478069356 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -353,7 +353,7 @@ void StorageNATS::read( } -SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageNATS::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto modified_context = addSettings(local_context); std::string subject = modified_context->getSettingsRef().stream_like_engine_insert_queue.changed diff --git a/src/Storages/NATS/StorageNATS.h b/src/Storages/NATS/StorageNATS.h index 518d81fb145..efe54243ee9 100644 --- a/src/Storages/NATS/StorageNATS.h +++ b/src/Storages/NATS/StorageNATS.h @@ -51,7 +51,7 @@ public: size_t /* max_block_size */, size_t /* num_streams */) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; /// We want to control the number of rows in a chunk inserted into NATS bool prefersLargeBlocks() const override { return false; } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index e84f5c963a8..651c63e1b91 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -764,7 +764,7 @@ void StorageRabbitMQ::read( } -SinkToStoragePtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageRabbitMQ::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto producer = std::make_unique( configuration, routing_keys, exchange_name, exchange_type, producer_id.fetch_add(1), persistent, shutdown_called, log); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index c531026d83a..dc410c4f298 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -57,7 +57,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; /// We want to control the number of rows in a chunk inserted into RabbitMQ bool prefersLargeBlocks() const override { return false; } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index d1195a9132e..27e8de78b0f 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -461,7 +461,7 @@ Pipe StorageEmbeddedRocksDB::read( } SinkToStoragePtr StorageEmbeddedRocksDB::write( - const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) { return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 32d7740009e..97fd07626a8 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -48,7 +48,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a4cb15d5711..d021667f771 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -656,7 +656,7 @@ private: }; -SinkToStoragePtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageBuffer::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) { return std::make_shared(*this, metadata_snapshot); } diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 83d2376216b..8f089a4d580 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -88,7 +88,7 @@ public: bool supportsSubcolumns() const override { return true; } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool /*async_insert*/) override; void startup() override; /// Flush all buffers into the subordinate table and stop background thread. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index bcf6f68d00d..d86e735f4b4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -897,7 +897,7 @@ QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, auto temporary_table_expression_node = std::make_shared(external_storage, mutable_context); temporary_table_expression_node->setTemporaryTableName(temporary_table_name); - auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context); + auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context, /*async_insert=*/false); auto io = interpreter.execute(); io.pipeline.complete(std::move(table_out)); CompletedPipelineExecutor executor(io.pipeline); @@ -1132,7 +1132,7 @@ void StorageDistributed::read( } -SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto cluster = getCluster(); const auto & settings = local_context->getSettingsRef(); diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 417fb6447bf..f45286341cf 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -118,7 +118,7 @@ public: bool supportsParallelInsert() const override { return true; } std::optional totalBytes(const Settings &) const override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool /*async_insert*/) override; std::optional distributedWrite(const ASTInsertQuery & query, ContextPtr context) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 052775aefca..72347789790 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1049,7 +1049,8 @@ private: SinkToStoragePtr StorageFile::write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) + ContextPtr context, + bool /*async_insert*/) { if (format_name == "Distributed") throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method write is not implemented for Distributed format"); diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 53ce7eeaaf6..9eb0d4b4383 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -50,7 +50,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; void truncate( const ASTPtr & /*query*/, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 5113320548d..a238e9ef26c 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -89,10 +89,10 @@ RWLockImpl::LockHolder StorageJoin::tryLockForCurrentQueryTimedWithContext(const return lock->getLock(type, query_id, acquire_timeout, false); } -SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { std::lock_guard mutate_lock(mutate_mutex); - return StorageSetOrJoinBase::write(query, metadata_snapshot, context); + return StorageSetOrJoinBase::write(query, metadata_snapshot, context, /*async_insert=*/false); } void StorageJoin::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context, TableExclusiveLockHolder &) diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index a5e85d8788a..5559b5d1ec8 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -59,7 +59,7 @@ public: /// (but not during processing whole query, it's safe for joinGet that doesn't involve `used_flags` from HashJoin) ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr context) const; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; Pipe read( const Names & column_names, diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index deebb9e0096..f2b1b907832 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -524,7 +524,7 @@ Pipe StorageKeeperMap::read( return process_keys(std::move(filtered_keys)); } -SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageKeeperMap::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { checkTable(); return std::make_shared(*this, metadata_snapshot->getSampleBlock(), local_context); diff --git a/src/Storages/StorageKeeperMap.h b/src/Storages/StorageKeeperMap.h index 552e6b35fe8..ad7b719e972 100644 --- a/src/Storages/StorageKeeperMap.h +++ b/src/Storages/StorageKeeperMap.h @@ -42,7 +42,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; void drop() override; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 02dc4843660..ac68de43332 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -855,7 +855,7 @@ Pipe StorageLog::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { WriteLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index a2b1356f240..f1d05ed39ac 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -55,7 +55,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index 9896265b576..08fbb61960f 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -32,7 +32,7 @@ public: QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr) override { throwNotAllowed(); } + SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, bool) override { throwNotAllowed(); } NamesAndTypesList getVirtuals() const override; ColumnSizeByName getColumnSizes() const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index b96c132d601..16d724d54d8 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -192,13 +192,13 @@ void StorageMaterializedView::read( } } -SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr local_context) +SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr local_context, bool async_insert) { auto storage = getTargetTable(); auto lock = storage->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto sink = storage->write(query, metadata_snapshot, local_context); + auto sink = storage->write(query, metadata_snapshot, local_context, async_insert); sink->addTableLock(lock); return sink; diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index f7876005c49..3ec789aa7e3 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -39,7 +39,7 @@ public: return target_table->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; void drop() override; void dropInnerTableIfAny(bool sync, ContextPtr local_context) override; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index caeefa5d96d..1b45b9ae3f4 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -159,7 +159,7 @@ void StorageMemory::read( } -SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageMemory::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { return std::make_shared(*this, metadata_snapshot, context); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index ce8a59b8bcd..c4f4331ca64 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -64,7 +64,7 @@ public: bool hasEvenlyDistributedRead() const override { return true; } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; void drop() override; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 3da4724471d..a2a46229660 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -274,7 +274,7 @@ std::optional StorageMergeTree::totalBytes(const Settings &) const } SinkToStoragePtr -StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { const auto & settings = local_context->getSettingsRef(); return std::make_shared( diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index be9f5426bbd..8099f9c16aa 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -71,7 +71,7 @@ public: std::optional totalRowsByPartitionPredicate(const SelectQueryInfo &, ContextPtr) const override; std::optional totalBytes(const Settings &) const override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; /** Perform the next step in combining the parts. */ diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 63b8c2d00a1..ab52fde3e3d 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -165,7 +165,7 @@ Pipe StorageMongoDB::read( return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size)); } -SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) +SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */, bool /*async_insert*/) { connectIfNotConnected(); return std::make_shared(collection_name, database_name, metadata_snapshot, connection); diff --git a/src/Storages/StorageMongoDB.h b/src/Storages/StorageMongoDB.h index 2b77f076e7e..36090d4584e 100644 --- a/src/Storages/StorageMongoDB.h +++ b/src/Storages/StorageMongoDB.h @@ -41,7 +41,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, - ContextPtr context) override; + ContextPtr context, + bool async_insert) override; struct Configuration { diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 2a96581d591..3e928c3a811 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -252,7 +252,7 @@ private: }; -SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { return std::make_shared( *this, diff --git a/src/Storages/StorageMySQL.h b/src/Storages/StorageMySQL.h index 9f47f9925d5..a98afc7ac4d 100644 --- a/src/Storages/StorageMySQL.h +++ b/src/Storages/StorageMySQL.h @@ -49,7 +49,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; struct Configuration { diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index d35c6a0b8b5..f6dd7064a22 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -46,7 +46,7 @@ public: bool supportsParallelInsert() const override { return true; } - SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr) override + SinkToStoragePtr write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, bool) override { return std::make_shared(metadata_snapshot->getSampleBlock()); } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5d7dc285c5d..3551ee36819 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -451,7 +451,7 @@ private: SinkToStoragePtr StoragePostgreSQL::write( - const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */) + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */, bool /*async_insert*/) { return std::make_shared(metadata_snapshot, pool->get(), remote_table_name, remote_table_schema, on_conflict); } diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index be6bbc5ec63..fb8b5a22df2 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -46,7 +46,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; struct Configuration { diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index e8a664a6382..14b7fc15af2 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -68,9 +68,9 @@ public: return getNested()->read(query_plan, column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override { - return getNested()->write(query, metadata_snapshot, context); + return getNested()->write(query, metadata_snapshot, context, async_insert); } void drop() override { getNested()->drop(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index db9c209a5fd..f9c9e958258 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4822,7 +4822,7 @@ void StorageReplicatedMergeTree::assertNotReadonly() const } -SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) { if (!initialization_done) throw Exception(ErrorCodes::NOT_INITIALIZED, "Table is not initialized yet"); @@ -4839,7 +4839,7 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con const auto storage_settings_ptr = getSettings(); const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; - bool async_deduplicate = query_settings.async_insert && query_settings.async_insert_deduplicate && storage_settings_ptr->replicated_deduplication_window_for_async_inserts != 0 && query_settings.insert_deduplicate; + bool async_deduplicate = async_insert && query_settings.async_insert_deduplicate && storage_settings_ptr->replicated_deduplication_window_for_async_inserts != 0 && query_settings.insert_deduplicate; if (async_deduplicate) return std::make_shared( *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index dd7ea84f76b..c0d9e36a8a7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -138,7 +138,7 @@ public: std::optional totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context) const override; std::optional totalBytes(const Settings & settings) const override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; std::optional distributedWrite(const ASTInsertQuery & /*query*/, ContextPtr /*context*/) override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f1a7bcb71a2..c3ed0f1af16 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1086,7 +1086,7 @@ Pipe StorageS3::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { auto query_configuration = updateConfigurationAndGetCopy(local_context); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index a4c120b99a6..9c2728c785d 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -293,7 +293,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index bd445217979..d5ae6f2383f 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -169,7 +169,7 @@ private: }; -SinkToStoragePtr StorageSQLite::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr) +SinkToStoragePtr StorageSQLite::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/, bool /*async_insert*/) { if (!sqlite_db) sqlite_db = openSQLiteDB(database_path, getContext(), /* throw_on_error */true); diff --git a/src/Storages/StorageSQLite.h b/src/Storages/StorageSQLite.h index 323c29ac8bb..9da040cbd5c 100644 --- a/src/Storages/StorageSQLite.h +++ b/src/Storages/StorageSQLite.h @@ -40,7 +40,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; static ColumnsDescription getTableStructureFromData( const SQLitePtr & sqlite_db_, diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 00b5dbfc5e3..f90539689e6 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -106,7 +106,7 @@ void SetOrJoinSink::onFinish() } -SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { UInt64 id = ++increment; return std::make_shared( diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index ccd1eb9912b..b310f817eb9 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -24,7 +24,7 @@ class StorageSetOrJoinBase : public IStorage public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {path}; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index b2e7c202800..5c704d877d1 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -394,7 +394,7 @@ Pipe StorageStripeLog::read( } -SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { WriteLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 3f1b4ed0ad5..f889a1de71b 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -49,7 +49,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool async_insert) override; void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index ccec087a8d9..26cbe1f0233 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -130,7 +130,8 @@ public: SinkToStoragePtr write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, - ContextPtr context) override + ContextPtr context, + bool async_insert) override { auto storage = getNested(); auto cached_structure = metadata_snapshot->getSampleBlock(); @@ -139,7 +140,7 @@ public: { throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Source storage and table function have different structure"); } - return storage->write(query, metadata_snapshot, context); + return storage->write(query, metadata_snapshot, context, async_insert); } void renameInMemory(const StorageID & new_table_id) override diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index efc44a069dd..520576d3961 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -803,7 +803,7 @@ Pipe StorageURLWithFailover::read( } -SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool /*async_insert*/) { if (http_method.empty()) http_method = Poco::Net::HTTPRequest::HTTP_POST; diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 316b142aec0..e80e19621e8 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -41,7 +41,7 @@ public: size_t max_block_size, size_t num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; bool supportsPartitionBy() const override { return true; } diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 9b3e203e337..b532d1c91f0 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -116,7 +116,7 @@ Pipe StorageXDBC::read( return IStorageURLBase::read(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams); } -SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) { bridge_helper->startBridgeSync(); diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index aa313e024ca..d7a1138c710 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -38,7 +38,7 @@ public: ContextPtr context_, BridgeHelperPtr bridge_helper_); - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; std::string getName() const override; private: diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 6ca74406b17..cef2feedcc5 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -215,7 +215,7 @@ void StorageSystemZooKeeper::read( query_plan.addStep(std::move(read_step)); } -SinkToStoragePtr StorageSystemZooKeeper::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context) +SinkToStoragePtr StorageSystemZooKeeper::write(const ASTPtr &, const StorageMetadataPtr &, ContextPtr context, bool /*async_insert*/) { if (!context->getConfigRef().getBool("allow_zookeeper_write", false)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Prohibit writing to system.zookeeper, unless config `allow_zookeeper_write` as true"); diff --git a/src/Storages/System/StorageSystemZooKeeper.h b/src/Storages/System/StorageSystemZooKeeper.h index c8988d787a0..a016d3ad74c 100644 --- a/src/Storages/System/StorageSystemZooKeeper.h +++ b/src/Storages/System/StorageSystemZooKeeper.h @@ -20,7 +20,7 @@ public: static NamesAndTypesList getNamesAndTypes(); - SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; + SinkToStoragePtr write(const ASTPtr & /*query*/, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/, bool /*async_insert*/) override; void read( QueryPlan & query_plan, diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 94d5db170a8..7fca9b5f078 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1549,7 +1549,7 @@ void StorageWindowView::writeIntoWindowView( auto lock = inner_table->lockForShare( local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = inner_table->getInMemoryMetadataPtr(); - auto output = inner_table->write(window_view.getMergeableQuery(), metadata_snapshot, local_context); + auto output = inner_table->write(window_view.getMergeableQuery(), metadata_snapshot, local_context, /*async_insert=*/false); output->addTableLock(lock); if (!blocksHaveEqualStructure(builder.getHeader(), output->getHeader())) diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index b63de6a66ef..352c44554bd 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -95,7 +95,7 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr con block.insert(column); } - QueryPipeline pipeline(table->write({}, metadata_snapshot, context)); + QueryPipeline pipeline(table->write({}, metadata_snapshot, context, /*async_insert=*/false)); PushingPipelineExecutor executor(pipeline); executor.push(block); diff --git a/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference new file mode 100644 index 00000000000..014be4ce1a9 --- /dev/null +++ b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.reference @@ -0,0 +1,17 @@ +0 +1 +1 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 diff --git a/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh new file mode 100755 index 00000000000..40e7c9feabf --- /dev/null +++ b/tests/queries/0_stateless/02784_disable_async_with_dedup_correctly.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS 02784_async_table_with_dedup" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE 02784_async_table_with_dedup (a Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/02784_async_table_with_dedup', 'r1') ORDER BY a" + +CLICKHOUSE_CLIENT_WITH_LOG=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g') + +function insert_with_log_check() { + $CLICKHOUSE_CLIENT_WITH_LOG --async-insert=1 --async_insert_deduplicate=1 --wait_for_async_insert=1 -q "$1" 2>&1 | grep -Fc "Setting async_insert=1, but INSERT query will be executed synchronously" +} + +insert_with_log_check "INSERT INTO 02784_async_table_with_dedup VALUES (1), (2)" +insert_with_log_check "INSERT INTO 02784_async_table_with_dedup SELECT number as a FROM system.numbers LIMIT 10 OFFSET 3" + +DATA_FILE=test_02784_async_$CLICKHOUSE_TEST_UNIQUE_NAME.csv +echo -e '13\n14' > $DATA_FILE + +insert_with_log_check "INSERT INTO 02784_async_table_with_dedup FROM INFILE '$DATA_FILE' FORMAT CSV" + +$CLICKHOUSE_CLIENT -q "SELECT a FROM 02784_async_table_with_dedup ORDER BY a" + +$CLICKHOUSE_CLIENT -q "DROP TABLE 02784_async_table_with_dedup" + +rm $DATA_FILE \ No newline at end of file From 129473ae744b7d8516e35ada5293e4b54be6f094 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 7 Jun 2023 20:48:03 +0200 Subject: [PATCH 150/159] Fix --- .../IO/CachedOnDiskWriteBufferFromFile.cpp | 30 ++++++++++--------- .../IO/CachedOnDiskWriteBufferFromFile.h | 4 +-- src/Interpreters/Cache/FileSegment.h | 6 ---- 3 files changed, 18 insertions(+), 22 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp index 9153af90312..b7727555480 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.cpp @@ -52,18 +52,20 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset FileSegment * file_segment; - if (file_segments.empty() || file_segments.back().isDownloaded()) + if (!file_segments || file_segments->empty() || file_segments->front().isDownloaded()) { file_segment = &allocateFileSegment(expected_write_offset, segment_kind); } else { - file_segment = &file_segments.back(); + file_segment = &file_segments->front(); } SCOPE_EXIT({ - if (file_segments.back().isDownloader()) - file_segments.back().completePartAndResetDownloader(); + if (!file_segments || file_segments->empty()) + return; + if (file_segments->front().isDownloader()) + file_segments->front().completePartAndResetDownloader(); }); while (size > 0) @@ -71,7 +73,7 @@ bool FileSegmentRangeWriter::write(const char * data, size_t size, size_t offset size_t available_size = file_segment->range().size() - file_segment->getDownloadedSize(false); if (available_size == 0) { - completeFileSegment(*file_segment); + completeFileSegment(); file_segment = &allocateFileSegment(expected_write_offset, segment_kind); continue; } @@ -114,10 +116,7 @@ void FileSegmentRangeWriter::finalize() if (finalized) return; - if (file_segments.empty()) - return; - - completeFileSegment(file_segments.back()); + completeFileSegment(); finalized = true; } @@ -145,10 +144,9 @@ FileSegment & FileSegmentRangeWriter::allocateFileSegment(size_t offset, FileSeg /// We set max_file_segment_size to be downloaded, /// if we have less size to write, file segment will be resized in complete() method. - auto holder = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings); - chassert(holder->size() == 1); - holder->moveTo(file_segments); - return file_segments.back(); + file_segments = cache->set(key, offset, cache->getMaxFileSegmentSize(), create_settings); + chassert(file_segments->size() == 1); + return file_segments->front(); } void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_segment) @@ -176,8 +174,12 @@ void FileSegmentRangeWriter::appendFilesystemCacheLog(const FileSegment & file_s cache_log->add(elem); } -void FileSegmentRangeWriter::completeFileSegment(FileSegment & file_segment) +void FileSegmentRangeWriter::completeFileSegment() { + if (!file_segments || file_segments->empty()) + return; + + auto & file_segment = file_segments->front(); /// File segment can be detached if space reservation failed. if (file_segment.isDetached() || file_segment.isCompleted()) return; diff --git a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h index 194afe88d88..8642886d6de 100644 --- a/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskWriteBufferFromFile.h @@ -43,7 +43,7 @@ private: void appendFilesystemCacheLog(const FileSegment & file_segment); - void completeFileSegment(FileSegment & file_segment); + void completeFileSegment(); FileCache * cache; FileSegment::Key key; @@ -53,7 +53,7 @@ private: String query_id; String source_path; - FileSegmentsHolder file_segments{}; + FileSegmentsHolderPtr file_segments; size_t expected_write_offset = 0; diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 163a15fcfda..186c65f12d9 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -360,12 +360,6 @@ struct FileSegmentsHolder : private boost::noncopyable FileSegments::const_iterator begin() const { return file_segments.begin(); } FileSegments::const_iterator end() const { return file_segments.end(); } - void moveTo(FileSegmentsHolder & holder) - { - holder.file_segments.insert(holder.file_segments.end(), file_segments.begin(), file_segments.end()); - file_segments.clear(); - } - private: FileSegments file_segments{}; const bool complete_on_dtor = true; From 92c87dedad043a17a1612f24b9fb43f175214b3f Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 8 Jun 2023 06:41:32 +0800 Subject: [PATCH 151/159] Add parallel state merge for some other combinator except If (#50413) * Add parallel state merge for some other combinator except If * add test * update test --- src/AggregateFunctions/AggregateFunctionArray.h | 7 +++++++ src/AggregateFunctions/AggregateFunctionMerge.h | 7 +++++++ src/AggregateFunctions/AggregateFunctionNull.h | 7 +++++++ src/AggregateFunctions/AggregateFunctionState.h | 7 +++++++ tests/performance/uniqExactIf.xml | 6 +++++- 5 files changed, 33 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionArray.h b/src/AggregateFunctions/AggregateFunctionArray.h index 21394e3ce05..7f38453f86b 100644 --- a/src/AggregateFunctions/AggregateFunctionArray.h +++ b/src/AggregateFunctions/AggregateFunctionArray.h @@ -141,6 +141,13 @@ public: nested_func->merge(place, rhs, arena); } + bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_func->merge(place, rhs, thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { nested_func->serialize(place, buf, version); diff --git a/src/AggregateFunctions/AggregateFunctionMerge.h b/src/AggregateFunctions/AggregateFunctionMerge.h index 0cb44259816..5b9e8e606af 100644 --- a/src/AggregateFunctions/AggregateFunctionMerge.h +++ b/src/AggregateFunctions/AggregateFunctionMerge.h @@ -110,6 +110,13 @@ public: nested_func->merge(place, rhs, arena); } + bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_func->merge(place, rhs, thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { nested_func->serialize(place, buf, version); diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index b817bad82fa..de7b190c949 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -148,6 +148,13 @@ public: nested_function->merge(nestedPlace(place), nestedPlace(rhs), arena); } + bool isAbleToParallelizeMerge() const override { return nested_function->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_function->merge(nestedPlace(place), nestedPlace(rhs), thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { bool flag = getFlag(place); diff --git a/src/AggregateFunctions/AggregateFunctionState.h b/src/AggregateFunctions/AggregateFunctionState.h index 625fe1f36bc..8335d21cb1e 100644 --- a/src/AggregateFunctions/AggregateFunctionState.h +++ b/src/AggregateFunctions/AggregateFunctionState.h @@ -91,6 +91,13 @@ public: nested_func->merge(place, rhs, arena); } + bool isAbleToParallelizeMerge() const override { return nested_func->isAbleToParallelizeMerge(); } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena * arena) const override + { + nested_func->merge(place, rhs, thread_pool, arena); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { nested_func->serialize(place, buf, version); diff --git a/tests/performance/uniqExactIf.xml b/tests/performance/uniqExactIf.xml index 409be257737..263fa75382c 100644 --- a/tests/performance/uniqExactIf.xml +++ b/tests/performance/uniqExactIf.xml @@ -1,3 +1,7 @@ - SELECT uniqExactIf(number, 1) FROM numbers_mt(1e6) + SELECT uniqExactIf(number, 1) FROM numbers_mt(1e7) + SELECT uniqExactState(number) FROM numbers_mt(1e7) Format Null + SELECT uniqExactArray([number]) FROM numbers_mt(1e7) Format Null + with (SELECT uniqExactState(number) FROM numbers_mt(1e7)) as a select uniqExactMerge(a) + SELECT uniqExactOrNull(number) FROM numbers_mt(1e7) From 286f3b247b33b08b17bd76320604d9b2b1c282b4 Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Thu, 8 Jun 2023 11:28:44 +0800 Subject: [PATCH 152/159] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8e2cd8d6027..3968751d5ee 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3155,7 +3155,7 @@ Possible values: - Positive integer. - 0 or 1 — Disabled. `SELECT` queries are executed in a single thread. -Default value: the number of physical CPU cores. +Default value: `max_threads`. ## opentelemetry_start_trace_probability {#opentelemetry-start-trace-probability} From 32d1acb3a57b0c1e942c64b7c434701f37fa1910 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Jun 2023 12:29:26 +0200 Subject: [PATCH 153/159] Add changelog for 23.5 --- CHANGELOG.md | 253 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 253 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1ccd4f9846d..f2ffdad9a7c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v23.5, 2023-06-08](#235)**
**[ClickHouse release v23.4, 2023-04-26](#234)**
**[ClickHouse release v23.3 LTS, 2023-03-30](#233)**
**[ClickHouse release v23.2, 2023-02-23](#232)**
@@ -7,6 +8,258 @@ # 2023 Changelog +### ClickHouse release 23.5, 2023-06-08 + +#### Upgrade Notes +* Compress marks and primary key by default. It significantly reduces the cold query time. Upgrade notes: the support for compressed marks and primary key has been added in version 22.9. If you turned on compressed marks or primary key or installed version 23.5 or newer, which has compressed marks or primary key on by default, you will not be able to downgrade to version 22.8 or earlier. You can also explicitly disable compressed marks or primary keys by specifying the `compress_marks` and `compress_primary_key` settings in the `` section of the server configuration file. **Upgrade notes:** If you upgrade from versions prior to 22.9, you should either upgrade all replicas at once or disable the compression before upgrade, or upgrade through an intermediate version, where the compressed marks are supported but not enabled by default, such as 23.3. [#42587](https://github.com/ClickHouse/ClickHouse/pull/42587) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make local object storage work consistently with s3 object storage, fix problem with append (closes [#48465](https://github.com/ClickHouse/ClickHouse/issues/48465)), make it configurable as independent storage. The change is backward incompatible because the cache on top of local object storage is not incompatible to previous versions. [#48791](https://github.com/ClickHouse/ClickHouse/pull/48791) ([Kseniia Sumarokova](https://github.com/kssenii)). +* The experimental feature "in-memory data parts" is removed. The data format is still supported, but the settings are no-op, and compact or wide parts will be used instead. This closes [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). [#49429](https://github.com/ClickHouse/ClickHouse/pull/49429) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Changed default values of settings `parallelize_output_from_storages` and `input_format_parquet_preserve_order`. This allows ClickHouse to reorder rows when reading from files (e.g. CSV or Parquet), greatly improving performance in many cases. To restore the old behavior of preserving order, use `parallelize_output_from_storages = 0`, `input_format_parquet_preserve_order = 1`. [#49479](https://github.com/ClickHouse/ClickHouse/pull/49479) ([Michael Kolupaev](https://github.com/al13n321)). +* Make projections production-ready. Add the `optimize_use_projections` setting to control whether the projections will be selected for SELECT queries. The setting `allow_experimental_projection_optimization` is obsolete and does nothing. [#49719](https://github.com/ClickHouse/ClickHouse/pull/49719) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Mark `joinGet` as non deterministic (so as `dictGet`). It allows using them in mutations without an extra setting. [#49843](https://github.com/ClickHouse/ClickHouse/pull/49843) ([Azat Khuzhin](https://github.com/azat)). +* Revert the "`groupArray` returns cannot be nullable" change (due to binary compatibility breakage for `groupArray`/`groupArrayLast`/`groupArraySample` over `Nullable` types, which likely will lead to `TOO_LARGE_ARRAY_SIZE` or `CANNOT_READ_ALL_DATA`). [#49971](https://github.com/ClickHouse/ClickHouse/pull/49971) ([Azat Khuzhin](https://github.com/azat)). +* Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. If you update from version prior to 22.12, we recommend to set this flag to `false` until update is finished. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). + +#### New Feature +* Added native ClickHouse Keeper CLI Client, it is available as `clickhouse keeper-client` [#47414](https://github.com/ClickHouse/ClickHouse/pull/47414) ([pufit](https://github.com/pufit)). +* Add `urlCluster` table function. Refactor all *Cluster table functions to reduce code duplication. Make schema inference work for all possible *Cluster function signatures and for named collections. Closes [#38499](https://github.com/ClickHouse/ClickHouse/issues/38499). [#45427](https://github.com/ClickHouse/ClickHouse/pull/45427) ([attack204](https://github.com/attack204)), Pavel Kruglov. +* The query cache can now be used for production workloads. [#47977](https://github.com/ClickHouse/ClickHouse/pull/47977) ([Robert Schulze](https://github.com/rschu1ze)). The query cache can now support queries with totals and extremes modifier. [#48853](https://github.com/ClickHouse/ClickHouse/pull/48853) ([Robert Schulze](https://github.com/rschu1ze)). Make `allow_experimental_query_cache` setting as obsolete for backward-compatibility. It was removed in https://github.com/ClickHouse/ClickHouse/pull/47977. [#49934](https://github.com/ClickHouse/ClickHouse/pull/49934) ([Timur Solodovnikov](https://github.com/tsolodov)). +* Geographical data types (`Point`, `Ring`, `Polygon`, and `MultiPolygon`) are production-ready. [#50022](https://github.com/ClickHouse/ClickHouse/pull/50022) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add schema inference to PostgreSQL, MySQL, MeiliSearch, and SQLite table engines. Closes [#49972](https://github.com/ClickHouse/ClickHouse/issues/49972). [#50000](https://github.com/ClickHouse/ClickHouse/pull/50000) ([Nikolay Degterinsky](https://github.com/evillique)). +* Password type in queries like `CREATE USER u IDENTIFIED BY 'p'` will be automatically set according to the setting `default_password_type` in the `config.xml` on the server. Closes [#42915](https://github.com/ClickHouse/ClickHouse/issues/42915). [#44674](https://github.com/ClickHouse/ClickHouse/pull/44674) ([Nikolay Degterinsky](https://github.com/evillique)). +* Add bcrypt password authentication type. Closes [#34599](https://github.com/ClickHouse/ClickHouse/issues/34599). [#44905](https://github.com/ClickHouse/ClickHouse/pull/44905) ([Nikolay Degterinsky](https://github.com/evillique)). +* Introduces new keyword `INTO OUTFILE 'file.txt' APPEND`. [#48880](https://github.com/ClickHouse/ClickHouse/pull/48880) ([alekar](https://github.com/alekar)). +* Added `system.zookeeper_connection` table that shows information about Keeper connections. [#45245](https://github.com/ClickHouse/ClickHouse/pull/45245) ([mateng915](https://github.com/mateng0915)). +* Add new function `generateRandomStructure` that generates random table structure. It can be used in combination with table function `generateRandom`. [#47409](https://github.com/ClickHouse/ClickHouse/pull/47409) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow the use of `CASE` without an `ELSE` branch and extended `transform` to deal with more types. Also fix some issues that made transform() return incorrect results when decimal types were mixed with other numeric types. [#48300](https://github.com/ClickHouse/ClickHouse/pull/48300) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Added [server-side encryption using KMS keys](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingKMSEncryption.html) with S3 tables, and the `header` setting with S3 disks. Closes [#48723](https://github.com/ClickHouse/ClickHouse/issues/48723). [#48724](https://github.com/ClickHouse/ClickHouse/pull/48724) ([Johann Gan](https://github.com/johanngan)). +* Add MemoryTracker for the background tasks (merges and mutation). Introduces `merges_mutations_memory_usage_soft_limit` and `merges_mutations_memory_usage_to_ram_ratio` settings that represent the soft memory limit for merges and mutations. If this limit is reached ClickHouse won't schedule new merge or mutation tasks. Also `MergesMutationsMemoryTracking` metric is introduced to allow observing current memory usage of background tasks. Resubmit [#46089](https://github.com/ClickHouse/ClickHouse/issues/46089). Closes [#48774](https://github.com/ClickHouse/ClickHouse/issues/48774). [#48787](https://github.com/ClickHouse/ClickHouse/pull/48787) ([Dmitry Novik](https://github.com/novikd)). +* Function `dotProduct` work for array. [#49050](https://github.com/ClickHouse/ClickHouse/pull/49050) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Support statement `SHOW INDEX` to improve compatibility with MySQL. [#49158](https://github.com/ClickHouse/ClickHouse/pull/49158) ([Robert Schulze](https://github.com/rschu1ze)). +* Add virtual column `_file` and `_path` support to table function `url`. - Impove error message for table function `url`. - resolves [#49231](https://github.com/ClickHouse/ClickHouse/issues/49231) - resolves [#49232](https://github.com/ClickHouse/ClickHouse/issues/49232). [#49356](https://github.com/ClickHouse/ClickHouse/pull/49356) ([Ziyi Tan](https://github.com/Ziy1-Tan)). +* Adding the `grants` field in the users.xml file, which allows specifying grants for users. [#49381](https://github.com/ClickHouse/ClickHouse/pull/49381) ([pufit](https://github.com/pufit)). +* Support full/right join by using grace hash join algorithm. [#49483](https://github.com/ClickHouse/ClickHouse/pull/49483) ([lgbo](https://github.com/lgbo-ustc)). +* `WITH FILL` modifier groups filling by sorting prefix. Controlled by `use_with_fill_by_sorting_prefix` setting (enabled by default). Related to [#33203](https://github.com/ClickHouse/ClickHouse/issues/33203)#issuecomment-1418736794. [#49503](https://github.com/ClickHouse/ClickHouse/pull/49503) ([Igor Nikonov](https://github.com/devcrafter)). +* Clickhouse-client now accepts queries after "--multiquery" when "--query" (or "-q") is absent. example: clickhouse-client --multiquery "select 1; select 2;". [#49870](https://github.com/ClickHouse/ClickHouse/pull/49870) ([Alexey Gerasimchuk](https://github.com/Demilivor)). +* Add separate `handshake_timeout` for receiving Hello packet from replica. Closes [#48854](https://github.com/ClickHouse/ClickHouse/issues/48854). [#49948](https://github.com/ClickHouse/ClickHouse/pull/49948) ([Kruglov Pavel](https://github.com/Avogar)). +* Added a function "space" which repeats a space as many times as specified. [#50103](https://github.com/ClickHouse/ClickHouse/pull/50103) ([Robert Schulze](https://github.com/rschu1ze)). +* Added --input_format_csv_trim_whitespaces option. [#50215](https://github.com/ClickHouse/ClickHouse/pull/50215) ([Alexey Gerasimchuk](https://github.com/Demilivor)). +* Allow the `dictGetAll` function for regexp tree dictionaries to return values from multiple matches as arrays. Closes [#50254](https://github.com/ClickHouse/ClickHouse/issues/50254). [#50255](https://github.com/ClickHouse/ClickHouse/pull/50255) ([Johann Gan](https://github.com/johanngan)). +* Added `toLastDayOfWeek` function to round a date or a date with time up to the nearest Saturday or Sunday. [#50315](https://github.com/ClickHouse/ClickHouse/pull/50315) ([Victor Krasnov](https://github.com/sirvickr)). +* Ability to ignore a skip index by specifying `ignore_data_skipping_indices`. [#50329](https://github.com/ClickHouse/ClickHouse/pull/50329) ([Boris Kuschel](https://github.com/bkuschel)). +* Add `system.user_processes` table and `SHOW USER PROCESSES` query to show memory info and ProfileEvents on user level. [#50492](https://github.com/ClickHouse/ClickHouse/pull/50492) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Add server and format settings `display_secrets_in_show_and_select` for displaying secrets of tables, databases, table functions, and dictionaries. Add privilege `displaySecretsInShowAndSelect` controlling which users can view secrets. [#46528](https://github.com/ClickHouse/ClickHouse/pull/46528) ([Mike Kot](https://github.com/myrrc)). +* Allow to set up a ROW POLICY for all tables that belong to a DATABASE. [#47640](https://github.com/ClickHouse/ClickHouse/pull/47640) ([Ilya Golshtein](https://github.com/ilejn)). + +#### Performance Improvement +* Compress marks and primary key by default. It significantly reduces the cold query time. Upgrade notes: the support for compressed marks and primary key has been added in version 22.9. If you turned on compressed marks or primary key or installed version 23.5 or newer, which has compressed marks or primary key on by default, you will not be able to downgrade to version 22.8 or earlier. You can also explicitly disable compressed marks or primary keys by specifying the `compress_marks` and `compress_primary_key` settings in the `` section of the server configuration file. [#42587](https://github.com/ClickHouse/ClickHouse/pull/42587) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* New setting s3_max_inflight_parts_for_one_file sets the limit of concurrently loaded parts with multipart upload request in scope of one file. [#49961](https://github.com/ClickHouse/ClickHouse/pull/49961) ([Sema Checherinda](https://github.com/CheSema)). +* When reading from multiple files reduce parallel parsing threads for each file. Resolves [#42192](https://github.com/ClickHouse/ClickHouse/issues/42192). [#46661](https://github.com/ClickHouse/ClickHouse/pull/46661) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Use aggregate projection only if it reads fewer granules than normal reading. It should help in case if query hits the PK of the table, but not the projection. Fixes [#49150](https://github.com/ClickHouse/ClickHouse/issues/49150). [#49417](https://github.com/ClickHouse/ClickHouse/pull/49417) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Do not store blocks in `ANY` hash join if nothing is inserted. [#48633](https://github.com/ClickHouse/ClickHouse/pull/48633) ([vdimir](https://github.com/vdimir)). +* Fixes aggregate combinator `-If` when JIT compiled, and enable JIT compilation for aggregate functions. Closes [#48120](https://github.com/ClickHouse/ClickHouse/issues/48120). [#49083](https://github.com/ClickHouse/ClickHouse/pull/49083) ([Igor Nikonov](https://github.com/devcrafter)). +* For reading from remote tables we use smaller tasks (instead of reading the whole part) to make tasks stealing work * task size is determined by size of columns to read * always use 1mb buffers for reading from s3 * boundaries of cache segments aligned to 1mb so they have decent size even with small tasks. it also should prevent fragmentation. [#49287](https://github.com/ClickHouse/ClickHouse/pull/49287) ([Nikita Taranov](https://github.com/nickitat)). +* Introduced settings: - `merge_max_block_size_bytes` to limit the amount of memory used for background operations. - `vertical_merge_algorithm_min_bytes_to_activate` to add another condition to activate vertical merges. [#49313](https://github.com/ClickHouse/ClickHouse/pull/49313) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Default size of a read buffer for reading from local filesystem changed to a slightly better value. Also two new settings are introduced: `max_read_buffer_size_local_fs` and `max_read_buffer_size_remote_fs`. [#49321](https://github.com/ClickHouse/ClickHouse/pull/49321) ([Nikita Taranov](https://github.com/nickitat)). +* Improve memory usage and speed of `SPARSE_HASHED`/`HASHED` dictionaries (e.g. `SPARSE_HASHED` now eats 2.6x less memory, and is ~2x faster). [#49380](https://github.com/ClickHouse/ClickHouse/pull/49380) ([Azat Khuzhin](https://github.com/azat)). +* Optimize the `system.query_log` and `system.query_thread_log` tables by applying `LowCardinality` when appropriate. The queries over these tables will be faster. [#49530](https://github.com/ClickHouse/ClickHouse/pull/49530) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Better performance when reading local `Parquet` files (through parallel reading). [#49539](https://github.com/ClickHouse/ClickHouse/pull/49539) ([Michael Kolupaev](https://github.com/al13n321)). +* Improve the performance of `RIGHT/FULL JOIN` by up to 2 times in certain scenarios, especially when joining a small left table with a large right table. [#49585](https://github.com/ClickHouse/ClickHouse/pull/49585) ([lgbo](https://github.com/lgbo-ustc)). +* Improve performance of BLAKE3 by 11% by enabling LTO for Rust. [#49600](https://github.com/ClickHouse/ClickHouse/pull/49600) ([Azat Khuzhin](https://github.com/azat)). Now it is on par with C++. +* Optimize the structure of the `system.opentelemetry_span_log`. Use `LowCardinality` where appropriate. Although this table is generally stupid (it is using the Map data type even for common attributes), it will be slightly better. [#49647](https://github.com/ClickHouse/ClickHouse/pull/49647) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Try to reserve hash table's size in `grace_hash` join. [#49816](https://github.com/ClickHouse/ClickHouse/pull/49816) ([lgbo](https://github.com/lgbo-ustc)). +* As is addresed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as `toYear`, `toYYYYMM`, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Parallel merge of `uniqExactIf` states. Closes [#49885](https://github.com/ClickHouse/ClickHouse/issues/49885). [#50285](https://github.com/ClickHouse/ClickHouse/pull/50285) ([flynn](https://github.com/ucasfl)). +* Keeper improvement: add `CheckNotExists` request to Keeper, which allows to improve the performance of Replicated tables. [#48897](https://github.com/ClickHouse/ClickHouse/pull/48897) ([Antonio Andelic](https://github.com/antonio2368)). +* Keeper performance improvements: avoid serializing same request twice while processing. Cache deserialization results of large requests. Controlled by new coordination setting `min_request_size_for_cache`. [#49004](https://github.com/ClickHouse/ClickHouse/pull/49004) ([Antonio Andelic](https://github.com/antonio2368)). +* Reduced number of `List` ZooKeeper requests when selecting parts to merge and a lot of partitions do not have anything to merge. [#49637](https://github.com/ClickHouse/ClickHouse/pull/49637) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Rework locking in the FS cache [#44985](https://github.com/ClickHouse/ClickHouse/pull/44985) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disable pure parallel replicas if trivial count optimization is possible. [#50594](https://github.com/ClickHouse/ClickHouse/pull/50594) ([Raúl Marín](https://github.com/Algunenano)). +* Don't send head request for all keys in Iceberg schema inference, only for keys that are used for reaing data. [#50203](https://github.com/ClickHouse/ClickHouse/pull/50203) ([Kruglov Pavel](https://github.com/Avogar)). +* Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). + +#### Experimental Feature +* `DEFLATE_QPL` codec lower the minimum simd version to SSE 4.2. [doc change in qpl](https://github.com/intel/qpl/commit/3f8f5cea27739f5261e8fd577dc233ffe88bf679) - Intel® QPL relies on a run-time kernels dispatcher and cpuid check to choose the best available implementation(sse/avx2/avx512) - restructured cmakefile for qpl build in clickhouse to align with latest upstream qpl. [#49811](https://github.com/ClickHouse/ClickHouse/pull/49811) ([jasperzhu](https://github.com/jinjunzh)). +* Add initial support to do JOINs with pure parallel replicas. [#49544](https://github.com/ClickHouse/ClickHouse/pull/49544) ([Raúl Marín](https://github.com/Algunenano)). +* More parallelism on `Outdated` parts removal with "zero-copy replication". [#49630](https://github.com/ClickHouse/ClickHouse/pull/49630) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Parallel Replicas: 1) Fixed an error `NOT_FOUND_COLUMN_IN_BLOCK` in case of using parallel replicas with non-replicated storage with disabled setting `parallel_replicas_for_non_replicated_merge_tree` 2) Now `allow_experimental_parallel_reading_from_replicas` have 3 possible values - 0, 1 and 2. 0 - disabled, 1 - enabled, silently disable them in case of failure (in case of FINAL or JOIN), 2 - enabled, throw an expection in case of failure. 3) If FINAL modifier is used in SELECT query and parallel replicas are enabled, ClickHouse will try to disable them if `allow_experimental_parallel_reading_from_replicas` is set to 1 and throw an exception otherwise. [#50195](https://github.com/ClickHouse/ClickHouse/pull/50195) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* When parallel replicas are enabled they will always skip unavailable servers (the behavior is controlled by the setting `skip_unavailable_shards`, enabled by default and can be only disabled). This closes: [#48565](https://github.com/ClickHouse/ClickHouse/issues/48565). [#50293](https://github.com/ClickHouse/ClickHouse/pull/50293) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Improvement +* The `BACKUP` command will not decrypt data from encrypted disks while making a backup. Instead the data will be stored in a backup in encrypted form. Such backups can be restored only to an encrypted disk with the same (or extended) list of encryption keys. [#48896](https://github.com/ClickHouse/ClickHouse/pull/48896) ([Vitaly Baranov](https://github.com/vitlibar)). +* Added possibility to use temporary tables in FROM part of ATTACH PARTITION FROM and REPLACE PARTITION FROM. [#49436](https://github.com/ClickHouse/ClickHouse/pull/49436) ([Roman Vasin](https://github.com/rvasin)). +* Added setting `async_insert` for `MergeTree` tables. It has the same meaning as query-level setting `async_insert` and enables asynchronous inserts for specific table. Note: it doesn't take effect for insert queries from `clickhouse-client`, use query-level setting in that case. [#49122](https://github.com/ClickHouse/ClickHouse/pull/49122) ([Anton Popov](https://github.com/CurtizJ)). +* Add support for size suffixes in quota creation statement parameters. [#49087](https://github.com/ClickHouse/ClickHouse/pull/49087) ([Eridanus](https://github.com/Eridanus117)). +* Extend `first_value` and `last_value` to accept NULL. [#46467](https://github.com/ClickHouse/ClickHouse/pull/46467) ([lgbo](https://github.com/lgbo-ustc)). +* Add alias `str_to_map` and `mapFromString` for `extractKeyValuePairs`. closes https://github.com/clickhouse/clickhouse/issues/47185. [#49466](https://github.com/ClickHouse/ClickHouse/pull/49466) ([flynn](https://github.com/ucasfl)). +* Add support for CGroup version 2 for asynchronous metrics about the memory usage and availability. This closes [#37983](https://github.com/ClickHouse/ClickHouse/issues/37983). [#45999](https://github.com/ClickHouse/ClickHouse/pull/45999) ([sichenzhao](https://github.com/sichenzhao)). +* Cluster table functions should always skip unavailable shards. close [#46314](https://github.com/ClickHouse/ClickHouse/issues/46314). [#46765](https://github.com/ClickHouse/ClickHouse/pull/46765) ([zk_kiger](https://github.com/zk-kiger)). +* Allow CSV file to contain empty columns in its header. [#47496](https://github.com/ClickHouse/ClickHouse/pull/47496) ([你不要过来啊](https://github.com/iiiuwioajdks)). +* Add Google Cloud Storage S3 compatible table function `gcs`. Like the `oss` and `cosn` functions, it is just an alias over the `s3` table function, and it does not bring any new features. [#47815](https://github.com/ClickHouse/ClickHouse/pull/47815) ([Kuba Kaflik](https://github.com/jkaflik)). +* Add ability to use strict parts size for S3 (compatibility with CloudFlare R2 S3 Storage). [#48492](https://github.com/ClickHouse/ClickHouse/pull/48492) ([Azat Khuzhin](https://github.com/azat)). +* Added new columns with info about `Replicated` database replicas to `system.clusters`: `database_shard_name`, `database_replica_name`, `is_active`. Added an optional `FROM SHARD` clause to `SYSTEM DROP DATABASE REPLICA` query. [#48548](https://github.com/ClickHouse/ClickHouse/pull/48548) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Add a new column `zookeeper_name` in system.replicas, to indicate on which (auxiliary) zookeeper cluster the replicated table's metadata is stored. [#48549](https://github.com/ClickHouse/ClickHouse/pull/48549) ([cangyin](https://github.com/cangyin)). +* `IN` operator support the comparison of `Date` and `Date32`. Closes [#48736](https://github.com/ClickHouse/ClickHouse/issues/48736). [#48806](https://github.com/ClickHouse/ClickHouse/pull/48806) ([flynn](https://github.com/ucasfl)). +* Support for erasure codes in `HDFS`, author: @M1eyu2018, @tomscut. [#48833](https://github.com/ClickHouse/ClickHouse/pull/48833) ([M1eyu](https://github.com/M1eyu2018)). +* Implement SYSTEM DROP REPLICA from auxillary ZooKeeper clusters, may be close [#48931](https://github.com/ClickHouse/ClickHouse/issues/48931). [#48932](https://github.com/ClickHouse/ClickHouse/pull/48932) ([wangxiaobo](https://github.com/wzb5212)). +* Add Array data type to MongoDB. Closes [#48598](https://github.com/ClickHouse/ClickHouse/issues/48598). [#48983](https://github.com/ClickHouse/ClickHouse/pull/48983) ([Nikolay Degterinsky](https://github.com/evillique)). +* Support storing `Interval` data types in tables. [#49085](https://github.com/ClickHouse/ClickHouse/pull/49085) ([larryluogit](https://github.com/larryluogit)). +* Allow using `ntile` window function without explicit window frame definition: `ntile(3) OVER (ORDER BY a)`, close [#46763](https://github.com/ClickHouse/ClickHouse/issues/46763). [#49093](https://github.com/ClickHouse/ClickHouse/pull/49093) ([vdimir](https://github.com/vdimir)). +* Added settings (`number_of_mutations_to_delay`, `number_of_mutations_to_throw`) to delay or throw `ALTER` queries that create mutations (`ALTER UPDATE`, `ALTER DELETE`, `ALTER MODIFY COLUMN`, ...) in case when table already has a lot of unfinished mutations. [#49117](https://github.com/ClickHouse/ClickHouse/pull/49117) ([Anton Popov](https://github.com/CurtizJ)). +* Catch exception from `create_directories` in filesystem cache. [#49203](https://github.com/ClickHouse/ClickHouse/pull/49203) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Copies embedded examples to a new field `example` in `system.functions` to supplement the field `description`. [#49222](https://github.com/ClickHouse/ClickHouse/pull/49222) ([Dan Roscigno](https://github.com/DanRoscigno)). +* Enable connection options for the MongoDB dictionary. Example: ``` xml localhost 27017 test dictionary_source ssl=true ``` ### Documentation entry for user-facing changes. [#49225](https://github.com/ClickHouse/ClickHouse/pull/49225) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Added an alias `asymptotic` for `asymp` computational method for `kolmogorovSmirnovTest`. Improved documentation. [#49286](https://github.com/ClickHouse/ClickHouse/pull/49286) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Aggregation function groupBitAnd/Or/Xor now work on signed integer data. This makes them consistent with the behavior of scalar functions bitAnd/Or/Xor. [#49292](https://github.com/ClickHouse/ClickHouse/pull/49292) ([exmy](https://github.com/exmy)). +* Split function-documentation into more fine-granular fields. [#49300](https://github.com/ClickHouse/ClickHouse/pull/49300) ([Robert Schulze](https://github.com/rschu1ze)). +* Use multiple threads shared between all tables within a server to load outdated data parts. The the size of the pool and its queue is controlled by `max_outdated_parts_loading_thread_pool_size` and `outdated_part_loading_thread_pool_queue_size` settings. [#49317](https://github.com/ClickHouse/ClickHouse/pull/49317) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Don't overestimate the size of processed data for `LowCardinality` columns when they share dictionaries between blocks. This closes [#49322](https://github.com/ClickHouse/ClickHouse/issues/49322). See also [#48745](https://github.com/ClickHouse/ClickHouse/issues/48745). [#49323](https://github.com/ClickHouse/ClickHouse/pull/49323) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Parquet writer now uses reasonable row group size when invoked through `OUTFILE`. [#49325](https://github.com/ClickHouse/ClickHouse/pull/49325) ([Michael Kolupaev](https://github.com/al13n321)). +* Allow restricted keywords like `ARRAY` as an alias if the alias is quoted. Closes [#49324](https://github.com/ClickHouse/ClickHouse/issues/49324). [#49360](https://github.com/ClickHouse/ClickHouse/pull/49360) ([Nikolay Degterinsky](https://github.com/evillique)). +* Data parts loading and deletion jobs were moved to shared server-wide pools instead of per-table pools. Pools sizes are controlled via settings `max_active_parts_loading_thread_pool_size`, `max_outdated_parts_loading_thread_pool_size` and `max_parts_cleaning_thread_pool_size` in top-level config. Table-level settings `max_part_loading_threads` and `max_part_removal_threads` became obsolete. [#49474](https://github.com/ClickHouse/ClickHouse/pull/49474) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow `?password=pass` in URL of the Play UI. Password is replaced in browser history. [#49505](https://github.com/ClickHouse/ClickHouse/pull/49505) ([Mike Kot](https://github.com/myrrc)). +* Allow reading zero-size objects from remote filesystems. (because empty files are not backup'd, so we might end up with zero blobs in metadata file). Closes [#49480](https://github.com/ClickHouse/ClickHouse/issues/49480). [#49519](https://github.com/ClickHouse/ClickHouse/pull/49519) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Attach thread MemoryTracker to `total_memory_tracker` after `ThreadGroup` detached. [#49527](https://github.com/ClickHouse/ClickHouse/pull/49527) ([Dmitry Novik](https://github.com/novikd)). +* Fix parameterized views when a query parameter is used multiple times in the query. [#49556](https://github.com/ClickHouse/ClickHouse/pull/49556) ([Azat Khuzhin](https://github.com/azat)). +* Release memory allocated for the last sent ProfileEvents snapshot in the context of a query. Followup [#47564](https://github.com/ClickHouse/ClickHouse/issues/47564). [#49561](https://github.com/ClickHouse/ClickHouse/pull/49561) ([Dmitry Novik](https://github.com/novikd)). +* Function "makeDate" now provides a MySQL-compatible overload (year & day of the year argument). [#49603](https://github.com/ClickHouse/ClickHouse/pull/49603) ([Robert Schulze](https://github.com/rschu1ze)). +* Support `dictionary` table function for `RegExpTreeDictionary`. [#49666](https://github.com/ClickHouse/ClickHouse/pull/49666) ([Han Fei](https://github.com/hanfei1991)). +* Added weighted fair IO scheduling policy. Added dynamic resource manager, which allows IO scheduling hierarchy to be updated in runtime w/o server restarts. [#49671](https://github.com/ClickHouse/ClickHouse/pull/49671) ([Sergei Trifonov](https://github.com/serxa)). +* Add compose request after multipart upload to GCS. This enables the usage of copy operation on objects uploaded with the multipart upload. It's recommended to set `s3_strict_upload_part_size` to some value because compose request can fail on objects created with parts of different sizes. [#49693](https://github.com/ClickHouse/ClickHouse/pull/49693) ([Antonio Andelic](https://github.com/antonio2368)). +* For the `extractKeyValuePairs` function: improve the "best-effort" parsing logic to accept `key_value_delimiter` as a valid part of the value. This also simplifies branching and might even speed up things a bit. [#49760](https://github.com/ClickHouse/ClickHouse/pull/49760) ([Arthur Passos](https://github.com/arthurpassos)). +* Add `initial_query_id` field for system.processors_profile_log [#49777](https://github.com/ClickHouse/ClickHouse/pull/49777) ([helifu](https://github.com/helifu)). +* System log tables can now have custom sorting keys. [#49778](https://github.com/ClickHouse/ClickHouse/pull/49778) ([helifu](https://github.com/helifu)). +* A new field `partitions` to `system.query_log` is used to indicate which partitions are participating in the calculation. [#49779](https://github.com/ClickHouse/ClickHouse/pull/49779) ([helifu](https://github.com/helifu)). +* Added `enable_the_endpoint_id_with_zookeeper_name_prefix` setting for `ReplicatedMergeTree` (disabled by default). When enabled, it adds ZooKeeper cluster name to table's interserver communication endpoint. It avoids `Duplicate interserver IO endpoint` errors when having replicated tables with the same path, but different auxiliary ZooKeepers. [#49780](https://github.com/ClickHouse/ClickHouse/pull/49780) ([helifu](https://github.com/helifu)). +* Add query parameters to `clickhouse-local`. Closes [#46561](https://github.com/ClickHouse/ClickHouse/issues/46561). [#49785](https://github.com/ClickHouse/ClickHouse/pull/49785) ([Nikolay Degterinsky](https://github.com/evillique)). +* Allow loading dictionaries and functions from YAML by default. In previous versions, it required editing the `dictionaries_config` or `user_defined_executable_functions_config` in the configuration file, as they expected `*.xml` files. [#49812](https://github.com/ClickHouse/ClickHouse/pull/49812) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The Kafka table engine now allows to use alias columns. [#49824](https://github.com/ClickHouse/ClickHouse/pull/49824) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Add setting to limit the max number of pairs produced by `extractKeyValuePairs`, a safeguard to avoid using way too much memory. [#49836](https://github.com/ClickHouse/ClickHouse/pull/49836) ([Arthur Passos](https://github.com/arthurpassos)). +* Add support for (an unusual) case where the arguments in the `IN` operator are single-element tuples. [#49844](https://github.com/ClickHouse/ClickHouse/pull/49844) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* `bitHammingDistance` function support `String` and `FixedString` data type. Closes [#48827](https://github.com/ClickHouse/ClickHouse/issues/48827). [#49858](https://github.com/ClickHouse/ClickHouse/pull/49858) ([flynn](https://github.com/ucasfl)). +* Fix timeout resetting errors in the client on OS X. [#49863](https://github.com/ClickHouse/ClickHouse/pull/49863) ([alekar](https://github.com/alekar)). +* Add support for big integers, such as UInt128, Int128, UInt256, and Int256 in the function `bitCount`. This enables Hamming distance over large bit masks for AI applications. [#49867](https://github.com/ClickHouse/ClickHouse/pull/49867) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fingerprints to be used instead of key IDs in encrypted disks. This simplifies the configuration of encrypted disks. [#49882](https://github.com/ClickHouse/ClickHouse/pull/49882) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add UUID data type to PostgreSQL. Closes [#49739](https://github.com/ClickHouse/ClickHouse/issues/49739). [#49894](https://github.com/ClickHouse/ClickHouse/pull/49894) ([Nikolay Degterinsky](https://github.com/evillique)). +* Function `toUnixTimestamp` now accepts `Date` and `Date32` arguments. [#49989](https://github.com/ClickHouse/ClickHouse/pull/49989) ([Victor Krasnov](https://github.com/sirvickr)). +* Charge only server memory for dictionaries. [#49995](https://github.com/ClickHouse/ClickHouse/pull/49995) ([Azat Khuzhin](https://github.com/azat)). +* The server will allow using the `SQL_*` settings such as `SQL_AUTO_IS_NULL` as no-ops for MySQL compatibility. This closes [#49927](https://github.com/ClickHouse/ClickHouse/issues/49927). [#50013](https://github.com/ClickHouse/ClickHouse/pull/50013) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Preserve initial_query_id for ON CLUSTER queries, which is useful for introspection (under `distributed_ddl_entry_format_version=5`). [#50015](https://github.com/ClickHouse/ClickHouse/pull/50015) ([Azat Khuzhin](https://github.com/azat)). +* Preserve backward incompatibility for renamed settings by using aliases (`allow_experimental_projection_optimization` for `optimize_use_projections`, `allow_experimental_lightweight_delete` for `enable_lightweight_delete`). [#50044](https://github.com/ClickHouse/ClickHouse/pull/50044) ([Azat Khuzhin](https://github.com/azat)). +* Support passing FQDN through setting my_hostname to register cluster node in keeper. Add setting of invisible to support multi compute groups. A compute group as a cluster, is invisible to other compute groups. [#50186](https://github.com/ClickHouse/ClickHouse/pull/50186) ([Yangkuan Liu](https://github.com/LiuYangkuan)). +* Fix PostgreSQL reading all the data even though `LIMIT n` could be specified. [#50187](https://github.com/ClickHouse/ClickHouse/pull/50187) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add new profile events for queries with subqueries (`QueriesWithSubqueries`/`SelectQueriesWithSubqueries`/`InsertQueriesWithSubqueries`). [#50204](https://github.com/ClickHouse/ClickHouse/pull/50204) ([Azat Khuzhin](https://github.com/azat)). +* Adding the roles field in the users.xml file, which allows specifying roles with grants via a config file. [#50278](https://github.com/ClickHouse/ClickHouse/pull/50278) ([pufit](https://github.com/pufit)). +* Report `CGroupCpuCfsPeriod` and `CGroupCpuCfsQuota` in AsynchronousMetrics. - Respect cgroup v2 memory limits during server startup. [#50379](https://github.com/ClickHouse/ClickHouse/pull/50379) ([alekar](https://github.com/alekar)). +* Add a signal handler for SIGQUIT to work the same way as SIGINT. Closes [#50298](https://github.com/ClickHouse/ClickHouse/issues/50298). [#50435](https://github.com/ClickHouse/ClickHouse/pull/50435) ([Nikolay Degterinsky](https://github.com/evillique)). +* In case JSON parse fails due to the large size of the object output the last position to allow debugging. [#50474](https://github.com/ClickHouse/ClickHouse/pull/50474) ([Valentin Alexeev](https://github.com/valentinalexeev)). +* Support decimals with not fixed size. Closes [#49130](https://github.com/ClickHouse/ClickHouse/issues/49130). [#50586](https://github.com/ClickHouse/ClickHouse/pull/50586) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Build/Testing/Packaging Improvement +* New and improved `keeper-bench`. Everything can be customized from YAML/XML file: - request generator - each type of request generator can have a specific set of fields - multi requests can be generated just by doing the same under `multi` key - for each request or subrequest in multi a `weight` field can be defined to control distribution - define trees that need to be setup for a test run - hosts can be defined with all timeouts customizable and it's possible to control how many sessions to generate for each host - integers defined with `min_value` and `max_value` fields are random number generators. [#48547](https://github.com/ClickHouse/ClickHouse/pull/48547) ([Antonio Andelic](https://github.com/antonio2368)). +* Io_uring is not supported on macos, don't choose it when running tests on local to avoid occassional failures. [#49250](https://github.com/ClickHouse/ClickHouse/pull/49250) ([Frank Chen](https://github.com/FrankChen021)). +* Support named fault injection for testing. [#49361](https://github.com/ClickHouse/ClickHouse/pull/49361) ([Han Fei](https://github.com/hanfei1991)). +* Allow running ClickHouse in the OS where the `prctl` (process control) syscall is not available, such as AWS Lambda. [#49538](https://github.com/ClickHouse/ClickHouse/pull/49538) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed the issue of build conflict between contrib/isa-l and isa-l in qpl [49296](https://github.com/ClickHouse/ClickHouse/issues/49296). [#49584](https://github.com/ClickHouse/ClickHouse/pull/49584) ([jasperzhu](https://github.com/jinjunzh)). +* Utilities are now only build if explicitly requested ("-DENABLE_UTILS=1") instead of by default, this reduces link times in typical development builds. [#49620](https://github.com/ClickHouse/ClickHouse/pull/49620) ([Robert Schulze](https://github.com/rschu1ze)). +* Pull build description of idxd-config into a separate CMake file to avoid accidental removal in future. [#49651](https://github.com/ClickHouse/ClickHouse/pull/49651) ([jasperzhu](https://github.com/jinjunzh)). +* Add CI check with an enabled analyzer in the master. Follow-up [#49562](https://github.com/ClickHouse/ClickHouse/issues/49562). [#49668](https://github.com/ClickHouse/ClickHouse/pull/49668) ([Dmitry Novik](https://github.com/novikd)). +* Switch to LLVM/clang 16. [#49678](https://github.com/ClickHouse/ClickHouse/pull/49678) ([Azat Khuzhin](https://github.com/azat)). +* Allow building ClickHouse with clang-17. [#49851](https://github.com/ClickHouse/ClickHouse/pull/49851) ([Alexey Milovidov](https://github.com/alexey-milovidov)). [#50410](https://github.com/ClickHouse/ClickHouse/pull/50410) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ClickHouse is now easier to be integrated into other cmake projects. [#49991](https://github.com/ClickHouse/ClickHouse/pull/49991) ([Amos Bird](https://github.com/amosbird)). (Which is strongly discouraged - Alexey Milovidov). +* Fix strange additional QEMU logging after [#47151](https://github.com/ClickHouse/ClickHouse/issues/47151), see https://s3.amazonaws.com/clickhouse-test-reports/50078/a4743996ee4f3583884d07bcd6501df0cfdaa346/stateless_tests__release__databasereplicated__[3_4].html. [#50442](https://github.com/ClickHouse/ClickHouse/pull/50442) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* ClickHouse can work on Linux RISC-V 6.1.22. This closes [#50456](https://github.com/ClickHouse/ClickHouse/issues/50456). [#50457](https://github.com/ClickHouse/ClickHouse/pull/50457) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Bump internal protobuf to v3.18 (fixes bogus CVE-2022-1941). [#50400](https://github.com/ClickHouse/ClickHouse/pull/50400) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump internal libxml2 to v2.10.4 (fixes bogus CVE-2023-28484 and bogus CVE-2023-29469). [#50402](https://github.com/ClickHouse/ClickHouse/pull/50402) ([Robert Schulze](https://github.com/rschu1ze)). +* Bump c-ares to v1.19.1 (bogus CVE-2023-32067, bogus CVE-2023-31130, bogus CVE-2023-31147). [#50403](https://github.com/ClickHouse/ClickHouse/pull/50403) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bogus CVE-2022-2469 in libgsasl. [#50404](https://github.com/ClickHouse/ClickHouse/pull/50404) ([Robert Schulze](https://github.com/rschu1ze)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* ActionsDAG: fix wrong optimization [#47584](https://github.com/ClickHouse/ClickHouse/pull/47584) ([Salvatore Mesoraca](https://github.com/aiven-sal)). +* Correctly handle concurrent snapshots in Keeper [#48466](https://github.com/ClickHouse/ClickHouse/pull/48466) ([Antonio Andelic](https://github.com/antonio2368)). +* MergeTreeMarksLoader holds DataPart instead of DataPartStorage [#48515](https://github.com/ClickHouse/ClickHouse/pull/48515) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Sequence state fix [#48603](https://github.com/ClickHouse/ClickHouse/pull/48603) ([Ilya Golshtein](https://github.com/ilejn)). +* Back/Restore concurrency check on previous fails [#48726](https://github.com/ClickHouse/ClickHouse/pull/48726) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix Attaching a table with non-existent ZK path does not increase the ReadonlyReplica metric [#48954](https://github.com/ClickHouse/ClickHouse/pull/48954) ([wangxiaobo](https://github.com/wzb5212)). +* Fix possible terminate called for uncaught exception in some places [#49112](https://github.com/ClickHouse/ClickHouse/pull/49112) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix key not found error for queries with multiple StorageJoin [#49137](https://github.com/ClickHouse/ClickHouse/pull/49137) ([vdimir](https://github.com/vdimir)). +* Fix wrong query result when using nullable primary key [#49172](https://github.com/ClickHouse/ClickHouse/pull/49172) ([Duc Canh Le](https://github.com/canhld94)). +* Fix reinterpretAs*() on big endian machines [#49198](https://github.com/ClickHouse/ClickHouse/pull/49198) ([Suzy Wang](https://github.com/SuzyWangIBMer)). +* (Experimental zero-copy replication) Lock zero copy parts more atomically [#49211](https://github.com/ClickHouse/ClickHouse/pull/49211) ([alesapin](https://github.com/alesapin)). +* Fix race on Outdated parts loading [#49223](https://github.com/ClickHouse/ClickHouse/pull/49223) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix all key value is null and group use rollup return wrong answer [#49282](https://github.com/ClickHouse/ClickHouse/pull/49282) ([Shuai li](https://github.com/loneylee)). +* Fix calculating load_factor for HASHED dictionaries with SHARDS [#49319](https://github.com/ClickHouse/ClickHouse/pull/49319) ([Azat Khuzhin](https://github.com/azat)). +* Disallow configuring compression CODECs for alias columns [#49363](https://github.com/ClickHouse/ClickHouse/pull/49363) ([Timur Solodovnikov](https://github.com/tsolodov)). +* Fix bug in removal of existing part directory [#49365](https://github.com/ClickHouse/ClickHouse/pull/49365) ([alesapin](https://github.com/alesapin)). +* Properly fix GCS when HMAC is used [#49390](https://github.com/ClickHouse/ClickHouse/pull/49390) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix fuzz bug when subquery set is not built when reading from remote() [#49425](https://github.com/ClickHouse/ClickHouse/pull/49425) ([Alexander Gololobov](https://github.com/davenger)). +* Invert `shutdown_wait_unfinished_queries` [#49427](https://github.com/ClickHouse/ClickHouse/pull/49427) ([Konstantin Bogdanov](https://github.com/thevar1able)). +* (Experimental zero-copy replication) Fix another zero copy bug [#49473](https://github.com/ClickHouse/ClickHouse/pull/49473) ([alesapin](https://github.com/alesapin)). +* Fix postgres database setting [#49481](https://github.com/ClickHouse/ClickHouse/pull/49481) ([Mal Curtis](https://github.com/snikch)). +* Correctly handle `s3Cluster` arguments [#49490](https://github.com/ClickHouse/ClickHouse/pull/49490) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix bug in TraceCollector destructor. [#49508](https://github.com/ClickHouse/ClickHouse/pull/49508) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix AsynchronousReadIndirectBufferFromRemoteFS breaking on short seeks [#49525](https://github.com/ClickHouse/ClickHouse/pull/49525) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix dictionaries loading order [#49560](https://github.com/ClickHouse/ClickHouse/pull/49560) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Forbid the change of data type of Object('json') column [#49563](https://github.com/ClickHouse/ClickHouse/pull/49563) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix stress test (Logical error: Expected 7134 >= 11030) [#49623](https://github.com/ClickHouse/ClickHouse/pull/49623) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix bug in DISTINCT [#49628](https://github.com/ClickHouse/ClickHouse/pull/49628) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix: DISTINCT in order with zero values in non-sorted columns [#49636](https://github.com/ClickHouse/ClickHouse/pull/49636) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix one-off error in big integers found by UBSan with fuzzer [#49645](https://github.com/ClickHouse/ClickHouse/pull/49645) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix reading from sparse columns after restart [#49660](https://github.com/ClickHouse/ClickHouse/pull/49660) ([Anton Popov](https://github.com/CurtizJ)). +* Fix assert in SpanHolder::finish() with fibers [#49673](https://github.com/ClickHouse/ClickHouse/pull/49673) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix short circuit functions and mutations with sparse arguments [#49716](https://github.com/ClickHouse/ClickHouse/pull/49716) ([Anton Popov](https://github.com/CurtizJ)). +* Fix writing appended files to incremental backups [#49725](https://github.com/ClickHouse/ClickHouse/pull/49725) ([Vitaly Baranov](https://github.com/vitlibar)). +* Ignore LWD column in checkPartDynamicColumns [#49737](https://github.com/ClickHouse/ClickHouse/pull/49737) ([Alexander Gololobov](https://github.com/davenger)). +* Fix msan issue in randomStringUTF8(uneven number) [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix aggregate function kolmogorovSmirnovTest [#49768](https://github.com/ClickHouse/ClickHouse/pull/49768) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). +* Fix settings aliases in native protocol [#49776](https://github.com/ClickHouse/ClickHouse/pull/49776) ([Azat Khuzhin](https://github.com/azat)). +* Fix `arrayMap` with array of tuples with single argument [#49789](https://github.com/ClickHouse/ClickHouse/pull/49789) ([Anton Popov](https://github.com/CurtizJ)). +* Fix per-query IO/BACKUPs throttling settings [#49797](https://github.com/ClickHouse/ClickHouse/pull/49797) ([Azat Khuzhin](https://github.com/azat)). +* Fix setting NULL in profile definition [#49831](https://github.com/ClickHouse/ClickHouse/pull/49831) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a bug with projections and the aggregate_functions_null_for_empty setting (for query_plan_optimize_projection) [#49873](https://github.com/ClickHouse/ClickHouse/pull/49873) ([Amos Bird](https://github.com/amosbird)). +* Fix processing pending batch for Distributed async INSERT after restart [#49884](https://github.com/ClickHouse/ClickHouse/pull/49884) ([Azat Khuzhin](https://github.com/azat)). +* Fix assertion in CacheMetadata::doCleanup [#49914](https://github.com/ClickHouse/ClickHouse/pull/49914) ([Kseniia Sumarokova](https://github.com/kssenii)). +* fix `is_prefix` in OptimizeRegularExpression [#49919](https://github.com/ClickHouse/ClickHouse/pull/49919) ([Han Fei](https://github.com/hanfei1991)). +* Fix metrics `WriteBufferFromS3Bytes`, `WriteBufferFromS3Microseconds` and `WriteBufferFromS3RequestsErrors` [#49930](https://github.com/ClickHouse/ClickHouse/pull/49930) ([Aleksandr Musorin](https://github.com/AVMusorin)). +* Fix IPv6 encoding in protobuf [#49933](https://github.com/ClickHouse/ClickHouse/pull/49933) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix possible Logical error on bad Nullable parsing for text formats [#49960](https://github.com/ClickHouse/ClickHouse/pull/49960) ([Kruglov Pavel](https://github.com/Avogar)). +* Add setting output_format_parquet_compliant_nested_types to produce more compatible Parquet files [#50001](https://github.com/ClickHouse/ClickHouse/pull/50001) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix logical error in stress test "Not enough space to add ..." [#50021](https://github.com/ClickHouse/ClickHouse/pull/50021) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Avoid deadlock when starting table in attach thread of `ReplicatedMergeTree` [#50026](https://github.com/ClickHouse/ClickHouse/pull/50026) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix assert in SpanHolder::finish() with fibers attempt 2 [#50034](https://github.com/ClickHouse/ClickHouse/pull/50034) ([Kruglov Pavel](https://github.com/Avogar)). +* Add proper escaping for DDL OpenTelemetry context serialization [#50045](https://github.com/ClickHouse/ClickHouse/pull/50045) ([Azat Khuzhin](https://github.com/azat)). +* Fix reporting broken projection parts [#50052](https://github.com/ClickHouse/ClickHouse/pull/50052) ([Amos Bird](https://github.com/amosbird)). +* JIT compilation not equals NaN fix [#50056](https://github.com/ClickHouse/ClickHouse/pull/50056) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix crashing in case of Replicated database without arguments [#50058](https://github.com/ClickHouse/ClickHouse/pull/50058) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash with `multiIf` and constant condition and nullable arguments [#50123](https://github.com/ClickHouse/ClickHouse/pull/50123) ([Anton Popov](https://github.com/CurtizJ)). +* Fix invalid index analysis for date related keys [#50153](https://github.com/ClickHouse/ClickHouse/pull/50153) ([Amos Bird](https://github.com/amosbird)). +* do not allow modify order by when there are no order by cols [#50154](https://github.com/ClickHouse/ClickHouse/pull/50154) ([Han Fei](https://github.com/hanfei1991)). +* Fix broken index analysis when binary operator contains a null constant argument [#50177](https://github.com/ClickHouse/ClickHouse/pull/50177) ([Amos Bird](https://github.com/amosbird)). +* clickhouse-client: disallow usage of `--query` and `--queries-file` at the same time [#50210](https://github.com/ClickHouse/ClickHouse/pull/50210) ([Alexey Gerasimchuk](https://github.com/Demilivor)). +* Fix UB for INTO OUTFILE extensions (APPEND / AND STDOUT) and WATCH EVENTS [#50216](https://github.com/ClickHouse/ClickHouse/pull/50216) ([Azat Khuzhin](https://github.com/azat)). +* Fix skipping spaces at end of row in CustomSeparatedIgnoreSpaces format [#50224](https://github.com/ClickHouse/ClickHouse/pull/50224) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix iceberg metadata parsing [#50232](https://github.com/ClickHouse/ClickHouse/pull/50232) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix nested distributed SELECT in WITH clause [#50234](https://github.com/ClickHouse/ClickHouse/pull/50234) ([Azat Khuzhin](https://github.com/azat)). +* Fix msan issue in keyed siphash [#50245](https://github.com/ClickHouse/ClickHouse/pull/50245) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bugs in Poco sockets in non-blocking mode, use true non-blocking sockets [#50252](https://github.com/ClickHouse/ClickHouse/pull/50252) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix checksum calculation for backup entries [#50264](https://github.com/ClickHouse/ClickHouse/pull/50264) ([Vitaly Baranov](https://github.com/vitlibar)). +* Comparison functions NaN fix [#50287](https://github.com/ClickHouse/ClickHouse/pull/50287) ([Maksim Kita](https://github.com/kitaisreal)). +* JIT aggregation nullable key fix [#50291](https://github.com/ClickHouse/ClickHouse/pull/50291) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix clickhouse-local crashing when writing empty Arrow or Parquet output [#50328](https://github.com/ClickHouse/ClickHouse/pull/50328) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix crash when Pool::Entry::disconnect() is called [#50334](https://github.com/ClickHouse/ClickHouse/pull/50334) ([Val Doroshchuk](https://github.com/valbok)). +* Improved fetch part by holding directory lock longer [#50339](https://github.com/ClickHouse/ClickHouse/pull/50339) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix bitShift* functions with both constant arguments [#50343](https://github.com/ClickHouse/ClickHouse/pull/50343) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix Keeper deadlock on exception when preprocessing requests. [#50387](https://github.com/ClickHouse/ClickHouse/pull/50387) ([frinkr](https://github.com/frinkr)). +* Fix hashing of const integer values [#50421](https://github.com/ClickHouse/ClickHouse/pull/50421) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix merge_tree_min_rows_for_seek/merge_tree_min_bytes_for_seek for data skipping indexes [#50432](https://github.com/ClickHouse/ClickHouse/pull/50432) ([Azat Khuzhin](https://github.com/azat)). +* Limit the number of in-flight tasks for loading outdated parts [#50450](https://github.com/ClickHouse/ClickHouse/pull/50450) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Keeper fix: apply uncommitted state after snapshot install [#50483](https://github.com/ClickHouse/ClickHouse/pull/50483) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix incorrect constant folding [#50536](https://github.com/ClickHouse/ClickHouse/pull/50536) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix logical error in stress test (Not enough space to add ...) [#50583](https://github.com/ClickHouse/ClickHouse/pull/50583) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix converting Null to LowCardinality(Nullable) in values table function [#50637](https://github.com/ClickHouse/ClickHouse/pull/50637) ([Kruglov Pavel](https://github.com/Avogar)). +* Revert invalid RegExpTreeDictionary optimization [#50642](https://github.com/ClickHouse/ClickHouse/pull/50642) ([Johann Gan](https://github.com/johanngan)). + ### ClickHouse release 23.4, 2023-04-26 #### Backward Incompatible Change From c37b80593c0db79b064bf5e54a817ef90238a343 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Thu, 8 Jun 2023 13:22:55 +0200 Subject: [PATCH 154/159] MaterializedMySQL: Add support of `TRUNCATE db.table` (#50624) Additional to `TRUNCATE TABLE db.table`. Co-authored-by: Alexander Tokmakov --- src/Parsers/MySQL/ASTDropQuery.cpp | 3 +- .../materialize_with_ddl.py | 40 +++++++++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/src/Parsers/MySQL/ASTDropQuery.cpp b/src/Parsers/MySQL/ASTDropQuery.cpp index fb76d93363a..890451e3e55 100644 --- a/src/Parsers/MySQL/ASTDropQuery.cpp +++ b/src/Parsers/MySQL/ASTDropQuery.cpp @@ -44,8 +44,9 @@ bool ParserDropQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & ex bool if_exists = false; bool is_truncate = false; - if (s_truncate.ignore(pos, expected) && s_table.ignore(pos, expected)) + if (s_truncate.ignore(pos, expected)) { + s_table.ignore(pos, expected); is_truncate = true; query->kind = ASTDropQuery::Kind::Table; ASTDropQuery::QualifiedName name; diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 97e2de49ceb..2bbbe9a3f13 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -379,6 +379,46 @@ def drop_table_with_materialized_mysql_database( "", ) + mysql_node.query( + "CREATE TABLE test_database_drop.test_table_3 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB" + ) + mysql_node.query("INSERT INTO test_database_drop.test_table_3 VALUES(1), (2)") + check_query( + clickhouse_node, + "SHOW TABLES FROM test_database_drop FORMAT TSV", + "test_table_2\ntest_table_3\n", + ) + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_3 ORDER BY id FORMAT TSV", + "1\n2\n", + ) + mysql_node.query("TRUNCATE test_database_drop.test_table_3") + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_3 ORDER BY id FORMAT TSV", + "", + ) + + mysql_node.query( + "CREATE TABLE test_database_drop.test_table_4 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB" + ) + mysql_node.query("INSERT INTO test_database_drop.test_table_4 VALUES(1), (2)") + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_4 ORDER BY id FORMAT TSV", + "1\n2\n", + ) + with mysql_node.alloc_connection() as mysql: + mysql.query("USE test_database_drop") + mysql.query("TRUNCATE test_table_4") + + check_query( + clickhouse_node, + "SELECT * FROM test_database_drop.test_table_4 ORDER BY id FORMAT TSV", + "", + ) + clickhouse_node.query("DROP DATABASE test_database_drop") mysql_node.query("DROP DATABASE test_database_drop") From 6b49816079016e362110da78e4e8b660894e9b6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Jun 2023 14:25:49 +0300 Subject: [PATCH 155/159] Update CHANGELOG.md Co-authored-by: Alexander Gololobov <440544+davenger@users.noreply.github.com> --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f2ffdad9a7c..959a7bad5dd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -210,7 +210,7 @@ * Fix assert in SpanHolder::finish() with fibers [#49673](https://github.com/ClickHouse/ClickHouse/pull/49673) ([Kruglov Pavel](https://github.com/Avogar)). * Fix short circuit functions and mutations with sparse arguments [#49716](https://github.com/ClickHouse/ClickHouse/pull/49716) ([Anton Popov](https://github.com/CurtizJ)). * Fix writing appended files to incremental backups [#49725](https://github.com/ClickHouse/ClickHouse/pull/49725) ([Vitaly Baranov](https://github.com/vitlibar)). -* Ignore LWD column in checkPartDynamicColumns [#49737](https://github.com/ClickHouse/ClickHouse/pull/49737) ([Alexander Gololobov](https://github.com/davenger)). +* Fix "There is no physical column _row_exists in table" error occurring during lightweight delete mutation on a table with Object column. [#49737](https://github.com/ClickHouse/ClickHouse/pull/49737) ([Alexander Gololobov](https://github.com/davenger)). * Fix msan issue in randomStringUTF8(uneven number) [#49750](https://github.com/ClickHouse/ClickHouse/pull/49750) ([Robert Schulze](https://github.com/rschu1ze)). * Fix aggregate function kolmogorovSmirnovTest [#49768](https://github.com/ClickHouse/ClickHouse/pull/49768) ([FFFFFFFHHHHHHH](https://github.com/FFFFFFFHHHHHHH)). * Fix settings aliases in native protocol [#49776](https://github.com/ClickHouse/ClickHouse/pull/49776) ([Azat Khuzhin](https://github.com/azat)). From c6b2ed234dcde2802020f730dc84101d7ad84f65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Jun 2023 13:27:13 +0200 Subject: [PATCH 156/159] Fix typos --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f2ffdad9a7c..c674af5852a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,7 +16,7 @@ * The experimental feature "in-memory data parts" is removed. The data format is still supported, but the settings are no-op, and compact or wide parts will be used instead. This closes [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). [#49429](https://github.com/ClickHouse/ClickHouse/pull/49429) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Changed default values of settings `parallelize_output_from_storages` and `input_format_parquet_preserve_order`. This allows ClickHouse to reorder rows when reading from files (e.g. CSV or Parquet), greatly improving performance in many cases. To restore the old behavior of preserving order, use `parallelize_output_from_storages = 0`, `input_format_parquet_preserve_order = 1`. [#49479](https://github.com/ClickHouse/ClickHouse/pull/49479) ([Michael Kolupaev](https://github.com/al13n321)). * Make projections production-ready. Add the `optimize_use_projections` setting to control whether the projections will be selected for SELECT queries. The setting `allow_experimental_projection_optimization` is obsolete and does nothing. [#49719](https://github.com/ClickHouse/ClickHouse/pull/49719) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Mark `joinGet` as non deterministic (so as `dictGet`). It allows using them in mutations without an extra setting. [#49843](https://github.com/ClickHouse/ClickHouse/pull/49843) ([Azat Khuzhin](https://github.com/azat)). +* Mark `joinGet` as non-deterministic (so as `dictGet`). It allows using them in mutations without an extra setting. [#49843](https://github.com/ClickHouse/ClickHouse/pull/49843) ([Azat Khuzhin](https://github.com/azat)). * Revert the "`groupArray` returns cannot be nullable" change (due to binary compatibility breakage for `groupArray`/`groupArrayLast`/`groupArraySample` over `Nullable` types, which likely will lead to `TOO_LARGE_ARRAY_SIZE` or `CANNOT_READ_ALL_DATA`). [#49971](https://github.com/ClickHouse/ClickHouse/pull/49971) ([Azat Khuzhin](https://github.com/azat)). * Setting `enable_memory_bound_merging_of_aggregation_results` is enabled by default. If you update from version prior to 22.12, we recommend to set this flag to `false` until update is finished. [#50319](https://github.com/ClickHouse/ClickHouse/pull/50319) ([Nikita Taranov](https://github.com/nickitat)). @@ -68,7 +68,7 @@ * Improve performance of BLAKE3 by 11% by enabling LTO for Rust. [#49600](https://github.com/ClickHouse/ClickHouse/pull/49600) ([Azat Khuzhin](https://github.com/azat)). Now it is on par with C++. * Optimize the structure of the `system.opentelemetry_span_log`. Use `LowCardinality` where appropriate. Although this table is generally stupid (it is using the Map data type even for common attributes), it will be slightly better. [#49647](https://github.com/ClickHouse/ClickHouse/pull/49647) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Try to reserve hash table's size in `grace_hash` join. [#49816](https://github.com/ClickHouse/ClickHouse/pull/49816) ([lgbo](https://github.com/lgbo-ustc)). -* As is addresed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as `toYear`, `toYYYYMM`, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* As is addressed in issue [#49748](https://github.com/ClickHouse/ClickHouse/issues/49748), the predicates with date converters, such as `toYear`, `toYYYYMM`, could be rewritten with the equivalent date (YYYY-MM-DD) comparisons at the AST level. And this transformation could bring performance improvement as it is free from the expensive date converter and the comparison between dates (or integers in the low level representation) is quite low-cost. The [prototype](https://github.com/ZhiguoZh/ClickHouse/commit/c7f1753f0c9363a19d95fa46f1cfed1d9f505ee0) shows that, with all identified date converters optimized, the overall QPS of the 13 queries is enhanced by **~11%** on the ICX server (Intel Xeon Platinum 8380 CPU, 80 cores, 160 threads). [#50062](https://github.com/ClickHouse/ClickHouse/pull/50062) [#50307](https://github.com/ClickHouse/ClickHouse/pull/50307) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). * Parallel merge of `uniqExactIf` states. Closes [#49885](https://github.com/ClickHouse/ClickHouse/issues/49885). [#50285](https://github.com/ClickHouse/ClickHouse/pull/50285) ([flynn](https://github.com/ucasfl)). * Keeper improvement: add `CheckNotExists` request to Keeper, which allows to improve the performance of Replicated tables. [#48897](https://github.com/ClickHouse/ClickHouse/pull/48897) ([Antonio Andelic](https://github.com/antonio2368)). * Keeper performance improvements: avoid serializing same request twice while processing. Cache deserialization results of large requests. Controlled by new coordination setting `min_request_size_for_cache`. [#49004](https://github.com/ClickHouse/ClickHouse/pull/49004) ([Antonio Andelic](https://github.com/antonio2368)). From 2aaaec2abebe4860120bd5db1cf6a1b38c28a12b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 8 Jun 2023 11:51:57 +0200 Subject: [PATCH 157/159] Catch issues with dockerd during the build --- tests/ci/build_check.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 4bc61c79fc0..35b98a7c3bb 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -343,6 +343,15 @@ def main(): f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True ) logging.info("Build finished with %s, log path %s", success, log_path) + if not success: + # We check if docker works, because if it's down, it's infrastructure + try: + subprocess.check_call("docker info", shell=True) + except subprocess.CalledProcessError: + logging.error( + "The dockerd looks down, won't upload anything and generate report" + ) + sys.exit(1) # FIXME performance performance_urls = [] From 6fcc4158d2ed912e0dcf34ca6e8f00060ec47dc0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Jun 2023 12:46:49 +0000 Subject: [PATCH 158/159] Add SHOW INDICES as alias for statement SHOW INDEX/INDEXES/KEYS --- docs/en/sql-reference/statements/show.md | 2 +- src/Parsers/ParserShowIndexesQuery.cpp | 2 +- tests/queries/0_stateless/02724_show_indexes.reference | 5 +++++ tests/queries/0_stateless/02724_show_indexes.sql | 1 + 4 files changed, 8 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 21c0010498a..f96eb55aa45 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -273,7 +273,7 @@ SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2 Displays a list of primary and data skipping indexes of a table. ```sql -SHOW [EXTENDED] {INDEX | INDEXES | KEYS } {FROM | IN} [{FROM | IN} ] [WHERE ] [INTO OUTFILE ] [FORMAT ] +SHOW [EXTENDED] {INDEX | INDEXES | INDICES | KEYS } {FROM | IN}
[{FROM | IN} ] [WHERE ] [INTO OUTFILE ] [FORMAT ] ``` The database and table name can be specified in abbreviated form as `.
`, i.e. `FROM tab FROM db` and `FROM db.tab` are diff --git a/src/Parsers/ParserShowIndexesQuery.cpp b/src/Parsers/ParserShowIndexesQuery.cpp index fe08b463069..7852ed9806d 100644 --- a/src/Parsers/ParserShowIndexesQuery.cpp +++ b/src/Parsers/ParserShowIndexesQuery.cpp @@ -28,7 +28,7 @@ bool ParserShowIndexesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (ParserKeyword("EXTENDED").ignore(pos, expected)) query->extended = true; - if (!(ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("INDEXES").ignore(pos, expected) || ParserKeyword("KEYS").ignore(pos, expected))) + if (!(ParserKeyword("INDEX").ignore(pos, expected) || ParserKeyword("INDEXES").ignore(pos, expected) || ParserKeyword("INDICES").ignore(pos, expected) || ParserKeyword("KEYS").ignore(pos, expected))) return false; if (ParserKeyword("FROM").ignore(pos, expected) || ParserKeyword("IN").ignore(pos, expected)) diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index 20af3954fa5..8365ade3231 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -14,6 +14,11 @@ tbl 0 mm1_idx \N \N \N \N \N \N \N minmax \N \N YES a, c, d tbl 0 mm2_idx \N \N \N \N \N \N \N minmax \N \N YES c, d, e tbl 0 PRIMARY \N \N A \N \N \N \N primary \N \N YES c, a tbl 0 set_idx \N \N \N \N \N \N \N set \N \N YES e +tbl 0 blf_idx \N \N \N \N \N \N \N bloom_filter \N \N YES d, b +tbl 0 mm1_idx \N \N \N \N \N \N \N minmax \N \N YES a, c, d +tbl 0 mm2_idx \N \N \N \N \N \N \N minmax \N \N YES c, d, e +tbl 0 PRIMARY \N \N A \N \N \N \N primary \N \N YES c, a +tbl 0 set_idx \N \N \N \N \N \N \N set \N \N YES e --- EXTENDED tbl 0 blf_idx \N \N \N \N \N \N \N bloom_filter \N \N YES d, b tbl 0 mm1_idx \N \N \N \N \N \N \N minmax \N \N YES a, c, d diff --git a/tests/queries/0_stateless/02724_show_indexes.sql b/tests/queries/0_stateless/02724_show_indexes.sql index ce8ed67c524..04a481fea4e 100644 --- a/tests/queries/0_stateless/02724_show_indexes.sql +++ b/tests/queries/0_stateless/02724_show_indexes.sql @@ -22,6 +22,7 @@ PRIMARY KEY (c, a); SELECT '--- Aliases of SHOW INDEX'; SHOW INDEX FROM tbl; SHOW INDEXES FROM tbl; +SHOW INDICES FROM tbl; SHOW KEYS FROM tbl; SELECT '--- EXTENDED'; From 5344ff2516d8f135c2a65f3979bf18109d76a10b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 8 Jun 2023 15:13:12 +0200 Subject: [PATCH 159/159] Temporarily disable annoy index tests (flaky for analyzer) (#50714) --- tests/queries/0_stateless/02354_annoy_index.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02354_annoy_index.sql b/tests/queries/0_stateless/02354_annoy_index.sql index 0168fa04c6f..abee5e8a6e4 100644 --- a/tests/queries/0_stateless/02354_annoy_index.sql +++ b/tests/queries/0_stateless/02354_annoy_index.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check +-- Tags: disabled, no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check SET allow_experimental_annoy_index = 1;