From 02f52bfbf2d80c1f6292bea7ad8ebad15412138f Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Thu, 3 Jun 2021 13:30:24 +0800 Subject: [PATCH 001/110] Add type check when create materialized view with to table --- src/Interpreters/InterpreterCreateQuery.cpp | 13 ++++++++++ ...ialized_view_to_table_type_check.reference | 5 ++++ ..._materialized_view_to_table_type_check.sql | 25 +++++++++++++++++++ 3 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/01880_materialized_view_to_table_type_check.reference create mode 100644 tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 86b810d031e..a341248da3c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -957,6 +957,19 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!created) /// Table already exists return {}; + /// Check type compatible for materialized dest table and select columns + if (create.select && create.is_materialized_view && create.to_table_id) + { + StoragePtr table = DatabaseCatalog::instance().getTable({create.database, create.table, create.uuid}, getContext()); + const auto & output_columns = table->getInMemoryMetadataPtr()->getSampleBlock(); + Block input_columns=InterpreterSelectWithUnionQuery( + create.select->clone(), getContext(),SelectQueryOptions().analyze()).getSampleBlock(); + auto actions_dag = ActionsDAG::makeConvertingActions( + input_columns.getColumnsWithTypeAndName(), + output_columns.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + } + return fillTableIfNeeded(create); } diff --git a/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.reference b/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.reference new file mode 100644 index 00000000000..5498a268179 --- /dev/null +++ b/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.reference @@ -0,0 +1,5 @@ +----------test--------: +----------test--------: +100 \0\0\0\0\0\0\0 +101 \0\0\0\0\0\0\0 +102 \0\0\0\0\0\0\0 diff --git a/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql b/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql new file mode 100644 index 00000000000..342ef08bc89 --- /dev/null +++ b/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS test_mv; +DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS test_input; + +CREATE TABLE test_input(id Int32) ENGINE=MergeTree() order by id; + +CREATE TABLE test(`id` Int32, `pv` AggregateFunction(sum, Int32)) ENGINE = AggregatingMergeTree() ORDER BY id; + +CREATE MATERIALIZED VIEW test_mv to test(`id` Int32, `pv` AggregateFunction(sum, Int32)) as SELECT id, sumState(1) as pv from test_input group by id; -- { serverError 70 } + +DROP VIEW test_mv; + +INSERT INTO test_input SELECT toInt32(number % 1000) AS id FROM numbers(10); +select '----------test--------:'; +select * from test; + +create MATERIALIZED VIEW test_mv to test(`id` Int32, `pv` AggregateFunction(sum, Int32)) as SELECT id, sumState(toInt32(1)) as pv from test_input group by id; +INSERT INTO test_input SELECT toInt32(number % 1000) AS id FROM numbers(100,3); + +select '----------test--------:'; +select * from test; + +DROP TABLE test_mv; +DROP TABLE test; +DROP TABLE test_input; From 4608d1ebde29797a2f34309a3f8c468bf9fdb494 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 8 Jun 2021 12:30:40 +0800 Subject: [PATCH 002/110] Fix error --- src/Interpreters/InterpreterCreateQuery.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a341248da3c..28230e0cba4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -962,12 +962,12 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { StoragePtr table = DatabaseCatalog::instance().getTable({create.database, create.table, create.uuid}, getContext()); const auto & output_columns = table->getInMemoryMetadataPtr()->getSampleBlock(); - Block input_columns=InterpreterSelectWithUnionQuery( - create.select->clone(), getContext(),SelectQueryOptions().analyze()).getSampleBlock(); + Block input_columns = InterpreterSelectWithUnionQuery( + create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); auto actions_dag = ActionsDAG::makeConvertingActions( input_columns.getColumnsWithTypeAndName(), output_columns.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); + ActionsDAG::MatchColumnsMode::Name); } return fillTableIfNeeded(create); From 21028ea93667f5a81072a6173e452c48cf1c33bd Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 15 Jun 2021 16:40:34 +0800 Subject: [PATCH 003/110] Add more type checks --- src/Interpreters/InterpreterCreateQuery.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 28230e0cba4..99315295514 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -960,13 +960,23 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Check type compatible for materialized dest table and select columns if (create.select && create.is_materialized_view && create.to_table_id) { - StoragePtr table = DatabaseCatalog::instance().getTable({create.database, create.table, create.uuid}, getContext()); - const auto & output_columns = table->getInMemoryMetadataPtr()->getSampleBlock(); + StoragePtr to_table = DatabaseCatalog::instance().getTable({create.to_table_id.database_name, + create.to_table_id.table_name, + create.to_table_id.uuid}, + getContext()); + const auto & to_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock(); + StoragePtr view_table = DatabaseCatalog::instance().getTable({create.database, create.table, create.uuid}, getContext()); + const auto & view_output_columns = view_table->getInMemoryMetadataPtr()->getSampleBlock(); + Block input_columns = InterpreterSelectWithUnionQuery( create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); - auto actions_dag = ActionsDAG::makeConvertingActions( + ActionsDAG::makeConvertingActions( input_columns.getColumnsWithTypeAndName(), - output_columns.getColumnsWithTypeAndName(), + to_output_columns.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + ActionsDAG::makeConvertingActions( + input_columns.getColumnsWithTypeAndName(), + view_output_columns.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); } From c6f182163ac6db6dd2d122ed752e5412d708c2a0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Apr 2022 10:57:39 +0300 Subject: [PATCH 004/110] Require mutations for per-table TTL only when it had been changed Before this patch only per-column TTL did not requires mutation if it had not been changed, after per-table TTL will also check whether it had been changed or not. Signed-off-by: Azat Khuzhin --- src/Storages/AlterCommands.cpp | 13 ++++++++--- ...per_table_ttl_mutation_on_change.reference | 22 +++++++++++++++++++ ...02265_per_table_ttl_mutation_on_change.sql | 22 +++++++++++++++++++ 3 files changed, 54 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.reference create mode 100644 tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index edd9dad2c02..5132fc9244c 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -818,22 +818,29 @@ bool AlterCommand::isCommentAlter() const bool AlterCommand::isTTLAlter(const StorageInMemoryMetadata & metadata) const { if (type == MODIFY_TTL) + { + if (!metadata.table_ttl.definition_ast) + return true; + /// If TTL had not been changed, do not require mutations + if (queryToString(metadata.table_ttl.definition_ast) == queryToString(ttl)) + return false; return true; + } if (!ttl || type != MODIFY_COLUMN) return false; - bool ttl_changed = true; + bool column_ttl_changed = true; for (const auto & [name, ttl_ast] : metadata.columns.getColumnTTLs()) { if (name == column_name && queryToString(*ttl) == queryToString(*ttl_ast)) { - ttl_changed = false; + column_ttl_changed = false; break; } } - return ttl_changed; + return column_ttl_changed; } bool AlterCommand::isRemovingProperty() const diff --git a/tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.reference b/tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.reference new file mode 100644 index 00000000000..740b4edf189 --- /dev/null +++ b/tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.reference @@ -0,0 +1,22 @@ +-- { echoOn } +alter table per_table_ttl_02265 modify TTL date + interval 1 month; +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +1 +alter table per_table_ttl_02265 modify TTL date + interval 1 month; +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +1 +alter table per_table_ttl_02265 modify TTL date + interval 2 month; +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +2 +alter table per_table_ttl_02265 modify TTL date + interval 2 month group by key set value = argMax(value, date); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +3 +alter table per_table_ttl_02265 modify TTL date + interval 2 month group by key set value = argMax(value, date); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +3 +alter table per_table_ttl_02265 modify TTL date + interval 2 month recompress codec(ZSTD(17)); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +4 +alter table per_table_ttl_02265 modify TTL date + interval 2 month recompress codec(ZSTD(17)); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +4 diff --git a/tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.sql b/tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.sql new file mode 100644 index 00000000000..53e2e72228a --- /dev/null +++ b/tests/queries/0_stateless/02265_per_table_ttl_mutation_on_change.sql @@ -0,0 +1,22 @@ +drop table if exists per_table_ttl_02265; +create table per_table_ttl_02265 (key Int, date Date, value String) engine=MergeTree() order by key; +insert into per_table_ttl_02265 values (1, today(), '1'); + +-- { echoOn } +alter table per_table_ttl_02265 modify TTL date + interval 1 month; +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +alter table per_table_ttl_02265 modify TTL date + interval 1 month; +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +alter table per_table_ttl_02265 modify TTL date + interval 2 month; +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +alter table per_table_ttl_02265 modify TTL date + interval 2 month group by key set value = argMax(value, date); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +alter table per_table_ttl_02265 modify TTL date + interval 2 month group by key set value = argMax(value, date); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +alter table per_table_ttl_02265 modify TTL date + interval 2 month recompress codec(ZSTD(17)); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; +alter table per_table_ttl_02265 modify TTL date + interval 2 month recompress codec(ZSTD(17)); +select count() from system.mutations where database = currentDatabase() and table = 'per_table_ttl_02265'; + +-- { echoOff } +drop table per_table_ttl_02265; From 8ebaf8498951bb9e866aa4f3ee4764e8146c5013 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 5 Apr 2022 13:39:05 +0300 Subject: [PATCH 005/110] Fix clang-tidy readability-simplify-boolean-expr warning in AlterCommands.cpp I want to make the code more readable before, but clang-tidy is too smart... Signed-off-by: Azat Khuzhin --- src/Storages/AlterCommands.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 5132fc9244c..286f58739f0 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -822,9 +822,7 @@ bool AlterCommand::isTTLAlter(const StorageInMemoryMetadata & metadata) const if (!metadata.table_ttl.definition_ast) return true; /// If TTL had not been changed, do not require mutations - if (queryToString(metadata.table_ttl.definition_ast) == queryToString(ttl)) - return false; - return true; + return queryToString(metadata.table_ttl.definition_ast) != queryToString(ttl); } if (!ttl || type != MODIFY_COLUMN) From 44bf8ba5681d237daf7721d648954581f2260c33 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Apr 2022 08:05:04 +0300 Subject: [PATCH 006/110] Remove trailing whitespace in default client_name Signed-off-by: Azat Khuzhin --- src/Interpreters/ClientInfo.cpp | 5 ++++- tests/queries/0_stateless/02270_client_name.reference | 1 + tests/queries/0_stateless/02270_client_name.sql | 3 +++ 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02270_client_name.reference create mode 100644 tests/queries/0_stateless/02270_client_name.sql diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 75af25e842e..614a966593f 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -193,7 +193,10 @@ void ClientInfo::setInitialQuery() { query_kind = QueryKind::INITIAL_QUERY; fillOSUserHostNameAndVersionInfo(); - client_name = (DBMS_NAME " ") + client_name; + if (client_name.empty()) + client_name = DBMS_NAME; + else + client_name = (DBMS_NAME " ") + client_name; } diff --git a/tests/queries/0_stateless/02270_client_name.reference b/tests/queries/0_stateless/02270_client_name.reference new file mode 100644 index 00000000000..fbb2921010e --- /dev/null +++ b/tests/queries/0_stateless/02270_client_name.reference @@ -0,0 +1 @@ +"ClickHouse" diff --git a/tests/queries/0_stateless/02270_client_name.sql b/tests/queries/0_stateless/02270_client_name.sql new file mode 100644 index 00000000000..8ec01090246 --- /dev/null +++ b/tests/queries/0_stateless/02270_client_name.sql @@ -0,0 +1,3 @@ +select 1 settings log_queries=1, log_queries_min_type='QUERY_FINISH' format Null; +system flush logs; +select client_name from system.query_log where current_database = currentDatabase() and query like 'select 1%' format CSV; From 50a32a74b70fbf7460bef948b14e742513b83020 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 8 Apr 2022 15:38:35 +0200 Subject: [PATCH 007/110] add thread_id and query_id to zookeeper_log --- src/Common/ZooKeeper/ZooKeeperCommon.h | 2 ++ src/Common/ZooKeeper/ZooKeeperImpl.cpp | 11 +++++++++++ src/Common/ZooKeeper/ZooKeeperImpl.h | 1 + src/Interpreters/ZooKeeperLog.cpp | 4 ++++ src/Interpreters/ZooKeeperLog.h | 2 ++ 5 files changed, 20 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index 80e38dd74d5..e51bea3f7f8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -57,6 +57,8 @@ struct ZooKeeperRequest : virtual Request bool restored_from_zookeeper_log = false; UInt64 request_created_time_ns = 0; + UInt64 thread_id = 0; + String query_id; ZooKeeperRequest() = default; ZooKeeperRequest(const ZooKeeperRequest &) = default; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 935df255843..b8a6e040bc8 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -1016,6 +1017,11 @@ void ZooKeeper::pushRequest(RequestInfo && info) try { info.time = clock::now(); + if (zk_log) + { + info.request->thread_id = getThreadId(); + info.request->query_id = String(CurrentThread::getQueryId()); + } if (!info.request->xid) { @@ -1269,6 +1275,11 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const elem.event_time = event_time; elem.address = socket_address; elem.session_id = session_id; + if (request) + { + elem.thread_id = request->thread_id; + elem.query_id = request->query_id; + } maybe_zk_log->add(elem); } } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 58c5947e8ea..faa777c33aa 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -219,6 +219,7 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; + UInt64 thread_id = 0; }; using RequestsQueue = ConcurrentBoundedQueue; diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index fdcbe430834..2828c3e2fa8 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -116,6 +116,8 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes() {"type", std::move(type_enum)}, {"event_date", std::make_shared()}, {"event_time", std::make_shared(6)}, + {"thread_id", std::make_shared()}, + {"query_id", std::make_shared()}, {"address", DataTypeFactory::instance().get("IPv6")}, {"port", std::make_shared()}, {"session_id", std::make_shared()}, @@ -164,6 +166,8 @@ void ZooKeeperLogElement::appendToBlock(MutableColumns & columns) const auto event_time_seconds = event_time / 1000000; columns[i++]->insert(DateLUT::instance().toDayNum(event_time_seconds).toUnderType()); columns[i++]->insert(event_time); + columns[i++]->insert(thread_id); + columns[i++]->insert(query_id); columns[i++]->insertData(IPv6ToBinary(address.host()).data(), 16); columns[i++]->insert(address.port()); columns[i++]->insert(session_id); diff --git a/src/Interpreters/ZooKeeperLog.h b/src/Interpreters/ZooKeeperLog.h index 284675a7ff5..733c61ba8fb 100644 --- a/src/Interpreters/ZooKeeperLog.h +++ b/src/Interpreters/ZooKeeperLog.h @@ -22,6 +22,8 @@ struct ZooKeeperLogElement Type type = UNKNOWN; Decimal64 event_time = 0; + UInt64 thread_id = 0; + String query_id; Poco::Net::SocketAddress address; Int64 session_id = 0; From 44b221fd22f9d8c0fa6f563d80e1dfafbf6726a8 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Fri, 8 Apr 2022 10:53:08 -0600 Subject: [PATCH 008/110] Updated adopters: added 4 new adopters --- docs/en/introduction/adopters.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 98eea85bbfa..e4d60a4fc9e 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -28,6 +28,7 @@ toc_title: Adopters | Badoo | Dating | Timeseries | — | 1.6 mln events/sec (2018) | [Slides in Russian, December 2019](https://presentations.clickhouse.com/meetup38/forecast.pdf) | | Beeline | Telecom | Data Platform | — | — | [Blog post, July 2021](https://habr.com/en/company/beeline/blog/567508/) | | Benocs | Network Telemetry and Analytics | Main Product | — | — | [Slides in English, October 2017](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup9/lpm.pdf) | +| Better Stack | Cloud, SaaS | Log Management | - | - | [Official Website](https://betterstack.com/logtail) | | BIGO | Video | Computing Platform | — | — | [Blog Article, August 2020](https://www.programmersought.com/article/44544895251/) | | BiliBili | Video sharing | — | — | — | [Blog post, June 2021](https://chowdera.com/2021/06/20210622012241476b.html) | | Bloomberg | Finance, Media | Monitoring | — | — | [Job opening, September 2021](https://careers.bloomberg.com/job/detail/94913), [slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | @@ -112,7 +113,7 @@ toc_title: Adopters | NLMK | Steel | Monitoring | — | — | [Article in Russian, Jan 2022](https://habr.com/en/company/nlmk/blog/645943/) | | NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | | Noction | Network Technology | Main Product | — | — | [Official Website](https://www.noction.com/news/irp-3-11-remote-triggered-blackholing-capability) -| ntop | Network Monitoning | Monitoring | — | — | [Official website, Jan 2022](https://www.ntop.org/ntop/historical-traffic-analysis-at-scale-using-clickhouse-with-ntopng/) | +| ntop | Network Monitoning | Monitoring | — | — | [Official website, January 2022](https://www.ntop.org/ntop/historical-traffic-analysis-at-scale-using-clickhouse-with-ntopng/) | | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | | Ok.ru | Social Network | — | 72 servers | 810 TB compressed, 50bn rows/day, 1.5 TB/day | [SmartData conference, October 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | | Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, October 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | @@ -123,6 +124,7 @@ toc_title: Adopters | Panelbear | Analytics | Monitoring and Analytics | — | — | [Tech Stack, November 2020](https://panelbear.com/blog/tech-stack/) | | Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | | Percona | Performance analysis | Percona Monitoring and Management | — | — | [Official website, Mar 2020](https://www.percona.com/blog/2020/03/30/advanced-query-analysis-in-percona-monitoring-and-management-with-direct-clickhouse-access/) | +| PingCAP | Analytics | Real-Time Transactional and Analytical Processing | - | - | [GitHub, TiFlash/TiDB](https://github.com/pingcap/tiflash) | | Plausible | Analytics | Main Product | — | — | [Blog post, June 2020](https://twitter.com/PlausibleHQ/status/1273889629087969280) | | PostHog | Product Analytics | Main Product | — | — | [Release Notes, October 2020](https://posthog.com/blog/the-posthog-array-1-15-0), [Blog, November 2021](https://posthog.com/blog/how-we-turned-clickhouse-into-our-eventmansion) | | Postmates | Delivery | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=188) | @@ -159,6 +161,7 @@ toc_title: Adopters | Suning | E-Commerce | User behaviour analytics | — | — | [Blog article](https://www.sohu.com/a/434152235_411876) | | Superwall | Monetization Tooling | Main product | — | — | [Word of mouth, Jan 2022](https://github.com/ClickHouse/ClickHouse/pull/33573) | | Swetrix | Analytics | Main Product | — | — | [Source code](https://github.com/swetrix/swetrix-api) | +| Synpse | Application Management | Main Product | - | - | [Tweet, January 2022](https://twitter.com/KRusenas/status/1483571168363880455) | | Teralytics | Mobility | Analytics | — | — | [Tech blog](https://www.teralytics.net/knowledge-hub/visualizing-mobility-data-the-scalability-challenge) | | Tencent | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | | Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | @@ -172,6 +175,7 @@ toc_title: Adopters | UTMSTAT | Analytics | Main product | — | — | [Blog post, June 2020](https://vc.ru/tribuna/133956-striming-dannyh-iz-servisa-skvoznoy-analitiki-v-clickhouse) | | Vercel | Traffic and Performance Analytics | — | — | — | Direct reference, October 2021 | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | +| VKontech | Distributed Systems | Migrating from MongoDB | - | - | [Blog, January 2022](https://vkontech.com/migrating-your-reporting-queries-from-a-general-purpose-db-mongodb-to-a-data-warehouse-clickhouse-performance-overview/) | | VMware | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | | Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | From 08dfce9a77ab6a025bd44e7a96492656da0553ec Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 8 Apr 2022 18:45:18 -0400 Subject: [PATCH 009/110] Prepared partitions differ from raw data In both the system exposed at `gh-api.clickhouse.com/play` and in the prepared partitions the field `IATA_CODE_Reporting_Airline` has been renamed as `Carrier`, which breaks many of the queries. Note: I assume that the raw data uses `IATA_CODE_Reporting_Airline`, but I do not have disk space to test. --- docs/en/getting-started/example-datasets/ontime.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/getting-started/example-datasets/ontime.md b/docs/en/getting-started/example-datasets/ontime.md index efc807b75fa..e4ae7237d57 100644 --- a/docs/en/getting-started/example-datasets/ontime.md +++ b/docs/en/getting-started/example-datasets/ontime.md @@ -159,6 +159,10 @@ $ clickhouse-client --query "select count(*) from datasets.ontime" !!! info "Info" If you will run the queries described below, you have to use the full table name, `datasets.ontime`. + +!!! info "Info" + If you are using the prepared partitions or the Online Playground replace any occurrence of `IATA_CODE_Reporting_Airline` or `IATA_CODE_Reporting_Airline AS Carrier` with Carrier (see `describe ontime`). + ## Queries {#queries} Q0. From 51dc21a55cd2ce1901b2e2fd191f78057147f611 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Fri, 8 Apr 2022 18:49:47 -0400 Subject: [PATCH 010/110] add ' --- docs/en/getting-started/example-datasets/ontime.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/ontime.md b/docs/en/getting-started/example-datasets/ontime.md index e4ae7237d57..87096354f1f 100644 --- a/docs/en/getting-started/example-datasets/ontime.md +++ b/docs/en/getting-started/example-datasets/ontime.md @@ -161,7 +161,7 @@ $ clickhouse-client --query "select count(*) from datasets.ontime" !!! info "Info" - If you are using the prepared partitions or the Online Playground replace any occurrence of `IATA_CODE_Reporting_Airline` or `IATA_CODE_Reporting_Airline AS Carrier` with Carrier (see `describe ontime`). + If you are using the prepared partitions or the Online Playground replace any occurrence of `IATA_CODE_Reporting_Airline` or `IATA_CODE_Reporting_Airline AS Carrier` in the following queries with `Carrier` (see `describe ontime`). ## Queries {#queries} From 3521292fad5ac69c05704c8c43520ac0c623966e Mon Sep 17 00:00:00 2001 From: jewisliu Date: Sun, 10 Apr 2022 11:54:15 +0800 Subject: [PATCH 011/110] Auto assign numbers for Enum elements --- src/DataTypes/DataTypeEnum.cpp | 20 +++++++++++++++++++ .../0_stateless/00757_enum_defaults.reference | 2 ++ .../0_stateless/00757_enum_defaults.sql | 6 ++++++ 3 files changed, 28 insertions(+) diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index b659d92e3f4..e292897e451 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -191,6 +191,24 @@ static void checkASTStructure(const ASTPtr & child) ErrorCodes::UNEXPECTED_AST_STRUCTURE); } +static void autoAssignNumberForEnum(const ASTPtr & arguments) +{ + Int16 child_count = 1; + ASTs assign_number_child; + assign_number_child.reserve(arguments->children.size()); + for (const ASTPtr & child : arguments->children) + { + if (child->as()) + { + ASTPtr func = makeASTFunction("equals", child, std::make_shared(child_count++)); + assign_number_child.emplace_back(func); + } + else + assign_number_child.emplace_back(child); + } + arguments->children = assign_number_child; +} + template static DataTypePtr createExact(const ASTPtr & arguments) { @@ -202,6 +220,7 @@ static DataTypePtr createExact(const ASTPtr & arguments) using FieldType = typename DataTypeEnum::FieldType; + autoAssignNumberForEnum(arguments); /// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument. for (const ASTPtr & child : arguments->children) { @@ -236,6 +255,7 @@ static DataTypePtr create(const ASTPtr & arguments) if (!arguments || arguments->children.empty()) throw Exception("Enum data type cannot be empty", ErrorCodes::EMPTY_DATA_PASSED); + autoAssignNumberForEnum(arguments); /// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument. for (const ASTPtr & child : arguments->children) { diff --git a/tests/queries/0_stateless/00757_enum_defaults.reference b/tests/queries/0_stateless/00757_enum_defaults.reference index 56ead34ad3b..35ef3c72406 100644 --- a/tests/queries/0_stateless/00757_enum_defaults.reference +++ b/tests/queries/0_stateless/00757_enum_defaults.reference @@ -4,3 +4,5 @@ iphone 1 iphone 1 \N 1 +a +b diff --git a/tests/queries/0_stateless/00757_enum_defaults.sql b/tests/queries/0_stateless/00757_enum_defaults.sql index 58f54a98b70..01f795b31f9 100644 --- a/tests/queries/0_stateless/00757_enum_defaults.sql +++ b/tests/queries/0_stateless/00757_enum_defaults.sql @@ -1,2 +1,8 @@ select os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; select toNullable(os_name) AS os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; + +DROP TABLE IF EXISTS auto_assgin_enum; +CREATE TABLE auto_assgin_enum (x enum('a', 'b')) ENGINE=MergeTree() order by x; +INSERT INTO auto_assgin_enum VALUES('a'), ('b'); +select * from auto_assgin_enum; +DROP TABLE auto_assgin_enum; From f0fad98a73b88ffb54e8f93228a2855bed15785e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Apr 2022 01:03:24 +0200 Subject: [PATCH 012/110] Reset thread name in thread pool --- src/Common/ThreadPool.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 8bfb93c9e94..15ae64f17fb 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -243,6 +244,9 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ while (true) { + /// This is inside the loop to also reset previous thread names set inside the jobs. + setThreadName("ThreadPoolIdle"); + Job job; bool need_shutdown = false; From 802f62a80e64bd6ba9ab2cbabf9d4fbea508443d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 11 Apr 2022 09:42:00 +0200 Subject: [PATCH 013/110] fix --- src/Common/ZooKeeper/ZooKeeperImpl.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index faa777c33aa..58c5947e8ea 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -219,7 +219,6 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; - UInt64 thread_id = 0; }; using RequestsQueue = ConcurrentBoundedQueue; From d2ccf876c6e68b78838e2c92078d4727f9e78c9b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Apr 2022 17:51:49 +0200 Subject: [PATCH 014/110] Fix --- src/Common/FileCache.cpp | 169 ++++++++++++------ src/Common/FileCache.h | 7 + src/Common/FileSegment.cpp | 42 ++++- src/Common/FileSegment.h | 3 + src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 26 ++- src/Disks/IO/CachedReadBufferFromRemoteFS.h | 2 + src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 4 +- src/IO/ReadBufferFromFileDescriptor.cpp | 2 +- 8 files changed, 191 insertions(+), 64 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 05d32f5ffe4..2f9eec33022 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -233,6 +233,88 @@ FileSegments LRUFileCache::splitRangeIntoCells( return file_segments; } +void LRUFileCache::fillHolesWithEmptyFileSegments( + FileSegments & file_segments, const Key & key, const FileSegment::Range & range, bool fill_with_detached_file_segments, std::lock_guard & cache_lock) +{ + /// There are segments [segment1, ..., segmentN] + /// (non-overlapping, non-empty, ascending-ordered) which (maybe partially) + /// intersect with given range. + + /// It can have holes: + /// [____________________] -- requested range + /// [____] [_] [_________] -- intersecting cache [segment1, ..., segmentN] + /// + /// For each such hole create a cell with file segment state EMPTY. + + auto it = file_segments.begin(); + auto segment_range = (*it)->range(); + + size_t current_pos; + if (segment_range.left < range.left) + { + /// [_______ -- requested range + /// [_______ + /// ^ + /// segment1 + + current_pos = segment_range.right + 1; + ++it; + } + else + current_pos = range.left; + + while (current_pos <= range.right && it != file_segments.end()) + { + segment_range = (*it)->range(); + + if (current_pos == segment_range.left) + { + current_pos = segment_range.right + 1; + ++it; + continue; + } + + assert(current_pos < segment_range.left); + + auto hole_size = segment_range.left - current_pos; + + if (fill_with_detached_file_segments) + { + auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY); + file_segment->detached = true; + file_segments.insert(it, file_segment); + } + else + { + file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, cache_lock)); + } + + current_pos = segment_range.right + 1; + ++it; + } + + if (current_pos <= range.right) + { + /// ________] -- requested range + /// _____] + /// ^ + /// segmentN + + auto hole_size = range.right - current_pos + 1; + + if (fill_with_detached_file_segments) + { + auto file_segment = std::make_shared(current_pos, hole_size, key, this, FileSegment::State::EMPTY); + file_segment->detached = true; + file_segments.insert(file_segments.end(), file_segment); + } + else + { + file_segments.splice(file_segments.end(), splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, cache_lock)); + } + } +} + FileSegmentsHolder LRUFileCache::getOrSet(const Key & key, size_t offset, size_t size) { assertInitialized(); @@ -254,69 +336,42 @@ FileSegmentsHolder LRUFileCache::getOrSet(const Key & key, size_t offset, size_t } else { - /// There are segments [segment1, ..., segmentN] - /// (non-overlapping, non-empty, ascending-ordered) which (maybe partially) - /// intersect with given range. - - /// It can have holes: - /// [____________________] -- requested range - /// [____] [_] [_________] -- intersecting cache [segment1, ..., segmentN] - /// - /// For each such hole create a cell with file segment state EMPTY. - - auto it = file_segments.begin(); - auto segment_range = (*it)->range(); - - size_t current_pos; - if (segment_range.left < range.left) - { - /// [_______ -- requested range - /// [_______ - /// ^ - /// segment1 - - current_pos = segment_range.right + 1; - ++it; - } - else - current_pos = range.left; - - while (current_pos <= range.right && it != file_segments.end()) - { - segment_range = (*it)->range(); - - if (current_pos == segment_range.left) - { - current_pos = segment_range.right + 1; - ++it; - continue; - } - - assert(current_pos < segment_range.left); - - auto hole_size = segment_range.left - current_pos; - file_segments.splice(it, splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, cache_lock)); - - current_pos = segment_range.right + 1; - ++it; - } - - if (current_pos <= range.right) - { - /// ________] -- requested range - /// _____] - /// ^ - /// segmentN - - auto hole_size = range.right - current_pos + 1; - file_segments.splice(file_segments.end(), splitRangeIntoCells(key, current_pos, hole_size, FileSegment::State::EMPTY, cache_lock)); - } + fillHolesWithEmptyFileSegments(file_segments, key, range, false, cache_lock); } assert(!file_segments.empty()); return FileSegmentsHolder(std::move(file_segments)); } +FileSegmentsHolder LRUFileCache::get(const Key & key, size_t offset, size_t size) +{ + assertInitialized(); + + FileSegment::Range range(offset, offset + size - 1); + + std::lock_guard cache_lock(mutex); + +#ifndef NDEBUG + assertCacheCorrectness(key, cache_lock); +#endif + + /// Get all segments which intersect with the given range. + auto file_segments = getImpl(key, range, cache_lock); + + if (file_segments.empty()) + { + auto file_segment = std::make_shared(offset, size, key, this, FileSegment::State::EMPTY); + file_segment->detached = true; + file_segments = { file_segment }; + } + else + { + fillHolesWithEmptyFileSegments(file_segments, key, range, true, cache_lock); + } + + return FileSegmentsHolder(std::move(file_segments)); +} + LRUFileCache::FileSegmentCell * LRUFileCache::addCell( const Key & key, size_t offset, size_t size, FileSegment::State state, std::lock_guard & cache_lock) diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index e706376bc89..b1d46569949 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -72,6 +72,8 @@ public: */ virtual FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) = 0; + virtual FileSegmentsHolder get(const Key & key, size_t offset, size_t size) = 0; + virtual FileSegmentsHolder setDownloading(const Key & key, size_t offset, size_t size) = 0; virtual FileSegments getSnapshot() const = 0; @@ -124,6 +126,8 @@ public: FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) override; + FileSegmentsHolder get(const Key & key, size_t offset, size_t size) override; + FileSegments getSnapshot() const override; FileSegmentsHolder setDownloading(const Key & key, size_t offset, size_t size) override; @@ -213,6 +217,9 @@ private: String dumpStructureImpl(const Key & key_, std::lock_guard & cache_lock); + void fillHolesWithEmptyFileSegments( + FileSegments & file_segments, const Key & key, const FileSegment::Range & range, bool fill_with_detached_file_segments, std::lock_guard & cache_lock); + public: struct Stat { diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 5a13ea7d207..42490146b37 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -107,6 +107,9 @@ String FileSegment::getOrSetDownloader() { std::lock_guard segment_lock(mutex); + if (detached) + throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Cannot set downloader for a detached file segment"); + if (downloader_id.empty()) { assert(download_state != State::DOWNLOADING); @@ -216,6 +219,8 @@ void FileSegment::write(const char * from, size_t size, size_t offset_) "Attempt to write {} bytes to offset: {}, but current download offset is {}", size, offset_, download_offset); + assertNotDetached(); + if (!cache_writer) { if (downloaded_size > 0) @@ -263,6 +268,8 @@ void FileSegment::writeInMemory(const char * from, size_t size) ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Not enough space is reserved. Available: {}, expected: {}", availableSize(), size); + assertNotDetached(); + std::lock_guard segment_lock(mutex); if (cache_writer) @@ -297,7 +304,9 @@ size_t FileSegment::finalizeWrite() size_t size = cache_writer->offset(); if (size == 0) - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing size is not allowed"); + throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Writing zero size is not allowed"); + + assertNotDetached(); try { @@ -352,6 +361,8 @@ bool FileSegment::reserve(size_t size) if (!size) throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "Zero space reservation is not allowed"); + assertNotDetached(); + { std::lock_guard segment_lock(mutex); @@ -419,7 +430,10 @@ void FileSegment::completeBatchAndResetDownloader() if (!is_downloader) { cv.notify_all(); - throw Exception(ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, "File segment can be completed only by downloader"); + throw Exception( + ErrorCodes::REMOTE_FS_OBJECT_CACHE_ERROR, + "File segment can be completed only by downloader ({} != {})", + downloader_id, getCallerId()); } resetDownloaderImpl(segment_lock); @@ -453,6 +467,8 @@ void FileSegment::complete(State state) download_state = state; + assertNotDetached(); + try { completeImpl(cache_lock, segment_lock); @@ -479,6 +495,8 @@ void FileSegment::complete(std::lock_guard & cache_lock) if (download_state != State::DOWNLOADED && getDownloadedSize(segment_lock) == range().size()) setDownloaded(segment_lock); + assertNotDetached(); + if (download_state == State::DOWNLOADING || download_state == State::EMPTY) { /// Segment state can be changed from DOWNLOADING or EMPTY only if the caller is the @@ -608,6 +626,12 @@ void FileSegment::assertCorrectnessImpl(std::lock_guard & /* segment assert(download_state != FileSegment::State::DOWNLOADED || std::filesystem::file_size(cache->getPathInLocalCache(key(), offset())) > 0); } +void FileSegment::assertNotDetached() const +{ + if (detached) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Operation not allowed, file segment is detached"); +} + FileSegmentPtr FileSegment::getSnapshot(const FileSegmentPtr & file_segment, std::lock_guard & /* cache_lock */) { auto snapshot = std::make_shared( @@ -638,6 +662,20 @@ FileSegmentsHolder::~FileSegmentsHolder() auto current_file_segment_it = file_segment_it; auto & file_segment = *current_file_segment_it; + if (file_segment->detached) + { +#ifndef NDEBUG + { + std::lock_guard cache_lock(file_segment->cache->mutex); + std::lock_guard segment_lock(file_segment->mutex); + bool is_last_holder = cache->isLastFileSegmentHolder(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); + assert(is_last_holder); + } +#endif + file_segment_it = file_segments.erase(current_file_segment_it); + continue; + } + if (!cache) cache = file_segment->cache; diff --git a/src/Common/FileSegment.h b/src/Common/FileSegment.h index 615fd9a56de..8439389fdeb 100644 --- a/src/Common/FileSegment.h +++ b/src/Common/FileSegment.h @@ -150,6 +150,7 @@ private: size_t getDownloadedSize(std::lock_guard & segment_lock) const; String getInfoForLogImpl(std::lock_guard & segment_lock) const; void assertCorrectnessImpl(std::lock_guard & segment_lock) const; + void assertNotDetached() const; void setDownloaded(std::lock_guard & segment_lock); void setDownloadFailed(std::lock_guard & segment_lock); @@ -199,6 +200,8 @@ private: Poco::Logger * log; + /// "detached" file segment means that it is not owned by cache ("detached" from cache). + /// In general case, all file segments are owned by cache. bool detached = false; std::atomic is_downloaded{false}; diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 16c1dd54f9f..d39d54970dc 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -46,7 +46,15 @@ CachedReadBufferFromRemoteFS::CachedReadBufferFromRemoteFS( void CachedReadBufferFromRemoteFS::initialize(size_t offset, size_t size) { - file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size)); + + if (settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) + { + file_segments_holder.emplace(cache->get(cache_key, offset, size)); + } + else + { + file_segments_holder.emplace(cache->getOrSet(cache_key, offset, size)); + } /** * Segments in returned list are ordered in ascending order and represent a full contiguous @@ -326,6 +334,10 @@ SeekableReadBufferPtr CachedReadBufferFromRemoteFS::getImplementationBuffer(File #endif size_t seek_offset = file_offset_of_buffer_end - range.left; + + if (file_offset_of_buffer_end < range.left) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} > {}", file_offset_of_buffer_end, range.left); + read_buffer_for_file_segment->seek(seek_offset, SEEK_SET); break; @@ -577,6 +589,8 @@ bool CachedReadBufferFromRemoteFS::nextImplStep() { last_caller_id = FileSegment::getCallerId(); + assertCorrectness(); + if (!initialized) initialize(file_offset_of_buffer_end, getTotalSizeToRead()); @@ -597,8 +611,8 @@ bool CachedReadBufferFromRemoteFS::nextImplStep() { try { - bool file_segment_already_completed = !file_segment->isDownloader(); - if (!file_segment_already_completed) + bool need_complete_file_segment = file_segment->isDownloader(); + if (need_complete_file_segment) file_segment->completeBatchAndResetDownloader(); } catch (...) @@ -820,6 +834,12 @@ std::optional CachedReadBufferFromRemoteFS::getLastNonDownloadedOffset() return std::nullopt; } +void CachedReadBufferFromRemoteFS::assertCorrectness() const +{ + if (IFileCache::isReadOnly() && !settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cache usage is not allowed"); +} + String CachedReadBufferFromRemoteFS::getInfoForLog() { String implementation_buffer_read_range_str; diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.h b/src/Disks/IO/CachedReadBufferFromRemoteFS.h index 5fc9ec39246..1e5b0a4a2d4 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.h +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.h @@ -50,6 +50,8 @@ private: bool nextImplStep(); + void assertCorrectness() const; + enum class ReadType { CACHED, diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 16a57b83771..e4690558b9c 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -38,7 +38,9 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S current_path = path; auto cache = settings.remote_fs_cache; - bool with_cache = cache && settings.enable_filesystem_cache; + bool with_cache = cache + && settings.enable_filesystem_cache + && (!IFileCache::isReadOnly() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache); auto remote_file_reader_creator = [=, this]() { diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index d266fb86e0f..f7006666b13 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -191,7 +191,7 @@ off_t ReadBufferFromFileDescriptor::seek(off_t offset, int whence) off_t res = ::lseek(fd, seek_pos, SEEK_SET); if (-1 == res) - throwFromErrnoWithPath("Cannot seek through file " + getFileName(), getFileName(), + throwFromErrnoWithPath(fmt::format("Cannot seek through file {} at offset {}", getFileName(), seek_pos), getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); /// Also note that seeking past the file size is not allowed. From dc314d53b3b56a0bd6a875e7b0b7ae1cd02c2bdc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 11 Apr 2022 20:28:11 -0400 Subject: [PATCH 015/110] do not create view before check, use MatchColumnsMode::Name mode --- src/Interpreters/InterpreterCreateQuery.cpp | 50 +++++++++++---------- 1 file changed, 27 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 02d4dfd5ea9..d8bbf5fcca6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1071,6 +1071,33 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create); + /// Check type compatible for materialized dest table and select columns + if (create.select && create.is_materialized_view && create.to_table_id) + { + StoragePtr to_table = DatabaseCatalog::instance().getTable({create.to_table_id.database_name, + create.to_table_id.table_name, + create.to_table_id.uuid}, + getContext()); + const auto & to_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock(); + + ColumnsWithTypeAndName view_output_columns; + for (const auto & [name, type] : properties.columns.getAllPhysical()) + view_output_columns.emplace_back(type, name); + + Block input_columns = InterpreterSelectWithUnionQuery( + create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); + + ActionsDAG::makeConvertingActions( + input_columns.getColumnsWithTypeAndName(), + view_output_columns, + ActionsDAG::MatchColumnsMode::Name); + + ActionsDAG::makeConvertingActions( + view_output_columns, + to_output_columns.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + } + DatabasePtr database; bool need_add_to_database = !create.temporary; if (need_add_to_database) @@ -1098,29 +1125,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!created) /// Table already exists return {}; - /// Check type compatible for materialized dest table and select columns - if (create.select && create.is_materialized_view && create.to_table_id) - { - StoragePtr to_table = DatabaseCatalog::instance().getTable({create.to_table_id.database_name, - create.to_table_id.table_name, - create.to_table_id.uuid}, - getContext()); - const auto & to_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - StoragePtr view_table = DatabaseCatalog::instance().getTable({create.database, create.table, create.uuid}, getContext()); - const auto & view_output_columns = view_table->getInMemoryMetadataPtr()->getSampleBlock(); - - Block input_columns = InterpreterSelectWithUnionQuery( - create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); - ActionsDAG::makeConvertingActions( - input_columns.getColumnsWithTypeAndName(), - to_output_columns.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); - ActionsDAG::makeConvertingActions( - input_columns.getColumnsWithTypeAndName(), - view_output_columns.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - } - /// If table has dependencies - add them to the graph QualifiedTableName qualified_name{database_name, create.getTable()}; TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); From e5494de63cdfd975a08f00469940193daf676f68 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 11 Apr 2022 21:07:45 -0700 Subject: [PATCH 016/110] h3Res0Indexes - remove unused array --- src/Functions/h3GetRes0Indexes.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Functions/h3GetRes0Indexes.cpp b/src/Functions/h3GetRes0Indexes.cpp index 7347b0fcf7f..9be55c0bb62 100644 --- a/src/Functions/h3GetRes0Indexes.cpp +++ b/src/Functions/h3GetRes0Indexes.cpp @@ -50,11 +50,8 @@ public: res0_indexes.resize(cell_count); getRes0Cells(res0_indexes.data()); - auto res = ColumnArray::create(ColumnUInt64::create()); - Array res_indexes; res_indexes.insert(res_indexes.end(), res0_indexes.begin(), res0_indexes.end()); - res->insert(res_indexes); return result_type->createColumnConst(input_rows_count, res_indexes); } From a77a228ab824e0c369aea03e999354169c91d241 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Apr 2022 09:37:03 +0200 Subject: [PATCH 017/110] Update FileSegment.cpp --- src/Common/FileSegment.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 42490146b37..2d0e55de518 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -662,11 +662,14 @@ FileSegmentsHolder::~FileSegmentsHolder() auto current_file_segment_it = file_segment_it; auto & file_segment = *current_file_segment_it; + if (!cache) + cache = file_segment->cache; + if (file_segment->detached) { #ifndef NDEBUG { - std::lock_guard cache_lock(file_segment->cache->mutex); + std::lock_guard cache_lock(cache->mutex); std::lock_guard segment_lock(file_segment->mutex); bool is_last_holder = cache->isLastFileSegmentHolder(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); assert(is_last_holder); @@ -676,9 +679,6 @@ FileSegmentsHolder::~FileSegmentsHolder() continue; } - if (!cache) - cache = file_segment->cache; - try { /// File segment pointer must be reset right after calling complete() and From 50b5684d004f311ad8f2917b1467da644cc600e0 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Apr 2022 11:59:58 +0200 Subject: [PATCH 018/110] Update FileSegment.cpp --- src/Common/FileSegment.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 2d0e55de518..92a478f3476 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -667,14 +667,6 @@ FileSegmentsHolder::~FileSegmentsHolder() if (file_segment->detached) { -#ifndef NDEBUG - { - std::lock_guard cache_lock(cache->mutex); - std::lock_guard segment_lock(file_segment->mutex); - bool is_last_holder = cache->isLastFileSegmentHolder(file_segment->key(), file_segment->offset(), cache_lock, segment_lock); - assert(is_last_holder); - } -#endif file_segment_it = file_segments.erase(current_file_segment_it); continue; } From b2c10611d15293e020526af7bd9c637686c7982c Mon Sep 17 00:00:00 2001 From: jewisliu Date: Mon, 11 Apr 2022 18:18:49 +0800 Subject: [PATCH 019/110] support UNSIGNED modifier with unused parameters of INT --- src/Parsers/ParserDataType.cpp | 15 +++++++++++++++ .../00841_temporary_table_database.reference | 4 ++++ .../00841_temporary_table_database.sql | 10 ++++++++++ 3 files changed, 29 insertions(+) diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 63a73ff4d1c..b0db5430726 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -97,6 +97,21 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) type_name_suffix = "SIGNED"; else if (ParserKeyword("UNSIGNED").ignore(pos)) type_name_suffix = "UNSIGNED"; + else if (pos->type == TokenType::OpeningRoundBracket) + { + ++pos; + if (pos->type != TokenType::Number) + return false; + ++pos; + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; + if (ParserKeyword("SIGNED").ignore(pos)) + type_name_suffix = "SIGNED"; + else if (ParserKeyword("UNSIGNED").ignore(pos)) + type_name_suffix = "UNSIGNED"; + } + } if (!type_name_suffix.empty()) diff --git a/tests/queries/0_stateless/00841_temporary_table_database.reference b/tests/queries/0_stateless/00841_temporary_table_database.reference index d00491fd7e5..f22f30a4b2a 100644 --- a/tests/queries/0_stateless/00841_temporary_table_database.reference +++ b/tests/queries/0_stateless/00841_temporary_table_database.reference @@ -1 +1,5 @@ 1 +CREATE TEMPORARY TABLE t3_00841\n(\n `x` UInt32\n)\nENGINE = Memory +1 +CREATE TEMPORARY TABLE t4_00841\n(\n `x` Int32\n)\nENGINE = Memory +1 diff --git a/tests/queries/0_stateless/00841_temporary_table_database.sql b/tests/queries/0_stateless/00841_temporary_table_database.sql index a5927a4cd33..96faa0e779b 100644 --- a/tests/queries/0_stateless/00841_temporary_table_database.sql +++ b/tests/queries/0_stateless/00841_temporary_table_database.sql @@ -3,3 +3,13 @@ INSERT INTO t1_00841 VALUES (1); SELECT * FROM t1_00841; CREATE TEMPORARY TABLE test.t2_00841 (x UInt8); -- { serverError 442 } + +CREATE TEMPORARY TABLE t3_00841 (x INT(11) UNSIGNED); +SHOW CREATE TEMPORARY TABLE t3_00841; +INSERT INTO t3_00841 VALUES (1); +SELECT * FROM t3_00841; + +CREATE TEMPORARY TABLE t4_00841 (x INT(11) SIGNED); +SHOW CREATE TEMPORARY TABLE t4_00841; +INSERT INTO t4_00841 VALUES (1); +SELECT * FROM t4_00841; From 457a9e9691f8330e334355244deebfa4a60c9295 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 12 Apr 2022 14:14:26 +0200 Subject: [PATCH 020/110] fixes for ReplicatedMergeTree --- src/Storages/MergeTree/DropPartsRanges.cpp | 6 +++- src/Storages/MergeTree/DropPartsRanges.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 24 +++++++-------- src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/MergeTree/MergeTreePartInfo.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 17 +++++++---- .../MergeTree/ReplicatedMergeTreeQueue.h | 3 ++ src/Storages/StorageReplicatedMergeTree.cpp | 30 ++++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 5 +--- 9 files changed, 52 insertions(+), 38 deletions(-) diff --git a/src/Storages/MergeTree/DropPartsRanges.cpp b/src/Storages/MergeTree/DropPartsRanges.cpp index a8dfd807d77..d467a7cac3d 100644 --- a/src/Storages/MergeTree/DropPartsRanges.cpp +++ b/src/Storages/MergeTree/DropPartsRanges.cpp @@ -51,12 +51,16 @@ void DropPartsRanges::removeDropRange(const ReplicatedMergeTreeLogEntryPtr & ent drop_ranges.erase(it); } -bool DropPartsRanges::hasDropRange(const MergeTreePartInfo & new_drop_range_info) const +bool DropPartsRanges::hasDropRange(const MergeTreePartInfo & new_drop_range_info, MergeTreePartInfo * out_drop_range_info) const { for (const auto & [_, drop_range] : drop_ranges) { if (drop_range.contains(new_drop_range_info)) + { + if (out_drop_range_info) + *out_drop_range_info = drop_range; return true; + } } return false; diff --git a/src/Storages/MergeTree/DropPartsRanges.h b/src/Storages/MergeTree/DropPartsRanges.h index 4d512263058..8cf25a406ee 100644 --- a/src/Storages/MergeTree/DropPartsRanges.h +++ b/src/Storages/MergeTree/DropPartsRanges.h @@ -30,7 +30,7 @@ public: bool isAffectedByDropRange(const std::string & new_part_name, std::string & postpone_reason) const; /// Already has equal DROP_RANGE. Don't need to assign new one - bool hasDropRange(const MergeTreePartInfo & new_drop_range_info) const; + bool hasDropRange(const MergeTreePartInfo & new_drop_range_info, MergeTreePartInfo * out_drop_range_info = nullptr) const; /// Add DROP_RANGE to map void addDropRange(const ReplicatedMergeTreeLogEntryPtr & entry); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a541822d6c1..e1e698d41e3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2886,7 +2886,7 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( if (drop_range.min_block > drop_range.max_block) throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid drop range: {}", drop_range.getPartName()); - auto partition_range = getDataPartsPartitionRange(drop_range.partition_id); + auto partition_range = getVisibleDataPartsVectorInPartition(txn, drop_range.partition_id, &lock); for (const DataPartPtr & part : partition_range) { @@ -2944,19 +2944,10 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( part->name, drop_range.getPartName()); } - if (part->getState() == DataPartState::Deleting) - continue; - - /// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice - if (txn) - { - if (!part->version.isVisible(*txn)) - continue; - } - parts_to_remove.emplace_back(part); } + /// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice removePartsFromWorkingSet(txn, parts_to_remove, clear_without_timeout, lock); return parts_to_remove; @@ -3388,13 +3379,18 @@ MergeTreeData::DataPartPtr MergeTreeData::getActiveContainingPart(const String & MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const { - if (const auto * txn = local_context->getCurrentTransaction().get()) + return getVisibleDataPartsVectorInPartition(local_context->getCurrentTransaction().get(), partition_id); +} + +MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartition(MergeTreeTransaction * txn, const String & partition_id, DataPartsLock * acquired_lock) const +{ + if (txn) { DataPartStateAndPartitionID active_parts{MergeTreeDataPartState::Active, partition_id}; DataPartStateAndPartitionID outdated_parts{MergeTreeDataPartState::Outdated, partition_id}; DataPartsVector res; { - auto lock = lockParts(); + auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); res.insert(res.end(), data_parts_by_state_and_info.lower_bound(active_parts), data_parts_by_state_and_info.upper_bound(active_parts)); res.insert(res.end(), data_parts_by_state_and_info.lower_bound(outdated_parts), data_parts_by_state_and_info.upper_bound(outdated_parts)); } @@ -3404,7 +3400,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartiti DataPartStateAndPartitionID state_with_partition{MergeTreeDataPartState::Active, partition_id}; - auto lock = lockParts(); + auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); return DataPartsVector( data_parts_by_state_and_info.lower_bound(state_with_partition), data_parts_by_state_and_info.upper_bound(state_with_partition)); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5ce5f30f0dc..3a099c471c2 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -490,6 +490,7 @@ public: void swapActivePart(MergeTreeData::DataPartPtr part_copy); /// Returns all parts in specified partition + DataPartsVector getVisibleDataPartsVectorInPartition(MergeTreeTransaction * txn, const String & partition_id, DataPartsLock * acquired_lock = nullptr) const; DataPartsVector getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const; DataPartsVector getVisibleDataPartsVectorInPartitions(ContextPtr local_context, const std::unordered_set & partition_ids) const; diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 065325920e2..e9ff6f87f0b 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -75,7 +75,7 @@ struct MergeTreePartInfo /// Return part mutation version, if part wasn't mutated return zero Int64 getMutationVersion() const { - return mutation ? mutation : 0; + return mutation; } /// True if parts do not intersect in any way. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c5798aaefe5..78ced7df822 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -94,6 +94,12 @@ bool ReplicatedMergeTreeQueue::isVirtualPart(const MergeTreeData::DataPartPtr & return !virtual_part_name.empty() && virtual_part_name != data_part->name; } +bool ReplicatedMergeTreeQueue::hasDropRange(const MergeTreePartInfo & part_info, MergeTreePartInfo * out_drop_range_info) const +{ + std::lock_guard lock(state_mutex); + return drop_ranges.hasDropRange(part_info, out_drop_range_info); +} + bool ReplicatedMergeTreeQueue::checkPartInQueueAndGetSourceParts(const String & part_name, Strings & source_parts) const { std::lock_guard lock(state_mutex); @@ -2141,21 +2147,21 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( if (pinned_part_uuids.part_uuids.contains(part->uuid)) { if (out_reason) - *out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; + *out_reason = fmt::format("Part {} has uuid {} which is currently pinned", part->name, part->uuid); return false; } if (part->name == inprogress_quorum_part) { if (out_reason) - *out_reason = "Quorum insert for part " + part->name + " is currently in progress"; + *out_reason = fmt::format("Quorum insert for part {} is currently in progress", part->name); return false; } if (prev_virtual_parts.getContainingPart(part->info).empty()) { if (out_reason) - *out_reason = "Entry for part " + part->name + " hasn't been read from the replication log yet"; + *out_reason = fmt::format("Entry for part {} hasn't been read from the replication log yet", part->name); return false; } @@ -2167,7 +2173,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( if (containing_part != part->name) { if (out_reason) - *out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; + *out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); return false; } @@ -2262,8 +2268,7 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const ReplicatedMerge bool ReplicatedMergeTreeMergePredicate::hasDropRange(const MergeTreePartInfo & new_drop_range_info) const { - std::lock_guard lock(queue.state_mutex); - return queue.drop_ranges.hasDropRange(new_drop_range_info); + return queue.hasDropRange(new_drop_range_info); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index ae0ca806344..0c0e872b0ac 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -400,6 +400,9 @@ public: /// Checks that part is already in virtual parts bool isVirtualPart(const MergeTreeData::DataPartPtr & data_part) const; + /// Returns true if part_info is covered by some DROP_RANGE + bool hasDropRange(const MergeTreePartInfo & part_info, MergeTreePartInfo * out_drop_range_info = nullptr) const; + /// Check that part produced by some entry in queue and get source parts for it. /// If there are several entries return largest source_parts set. This rarely possible /// for example after replica clone. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 66a5baf555b..0e61d8b6f81 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1812,7 +1812,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che if (!parts_for_merge.empty() && replica.empty()) { - LOG_INFO(log, "No active replica has part {}. Will fetch merged part instead.", entry.new_part_name); + LOG_INFO(log, " {}. Will fetch merged part instead.", entry.new_part_name); /// We should enqueue it for check, because merged part may never appear if source part is lost enqueuePartForCheck(entry.new_part_name); return false; @@ -6984,42 +6984,39 @@ bool StorageReplicatedMergeTree::dropPartImpl( if (!part) { if (throw_if_noop) - throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} not found locally, won't try to drop it.", part_name); return false; } if (merge_pred.hasDropRange(part->info)) { if (throw_if_noop) - throw Exception("Already has DROP RANGE for part " + part_name + " in queue.", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, "Already has DROP RANGE for part {} in queue.", part_name); return false; } /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already /// finished the merge. - if (partIsAssignedToBackgroundOperation(part)) + String out_reason; + if (!merge_pred.canMergeSinglePart(part, &out_reason)) { if (throw_if_noop) - throw Exception("Part " + part_name - + " is currently participating in a background operation (mutation/merge)" - + ", try again later", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + throw Exception(ErrorCodes::PART_IS_TEMPORARILY_LOCKED, out_reason); return false; } if (partIsLastQuorumPart(part->info)) { if (throw_if_noop) - throw Exception("Part " + part_name + " is last inserted part with quorum in partition. Cannot drop", - ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part {} is last inserted part with quorum in partition. Cannot drop", part_name); return false; } if (partIsInsertingWithParallelQuorum(part->info)) { if (throw_if_noop) - throw Exception("Part " + part_name + " is inserting with parallel quorum. Cannot drop", - ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Part {} is inserting with parallel quorum. Cannot drop", part_name); return false; } @@ -7146,6 +7143,17 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition( "Cannot assign ALTER PARTITION because another ALTER PARTITION query was concurrently executed"); } +void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds) +{ + MergeTreePartInfo covering_drop_range; + if (queue.hasDropRange(MergeTreePartInfo::fromPartName(part_name, format_version), &covering_drop_range)) + { + LOG_WARNING(log, "Do not enqueue part {} for check because it's covered by DROP_RANGE {} and going to be removed", + part_name, covering_drop_range.getPartName()); + return; + } + part_check_thread.enqueuePart(part_name, delay_to_check_seconds); +} CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 317544c8bb8..59fb3f124c7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -200,10 +200,7 @@ public: void getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay); /// Add a part to the queue of parts whose data you want to check in the background thread. - void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0) - { - part_check_thread.enqueuePart(part_name, delay_to_check_seconds); - } + void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0); CheckResults checkData(const ASTPtr & query, ContextPtr context) override; From 25101de7e2cd081e2a318cb5c7a46e08048ef731 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 12 Apr 2022 09:29:32 -0400 Subject: [PATCH 021/110] check only selected columns against destination table --- src/Interpreters/InterpreterCreateQuery.cpp | 33 ++++++++++----------- 1 file changed, 15 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d8bbf5fcca6..9a7f618696a 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1074,28 +1074,25 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// Check type compatible for materialized dest table and select columns if (create.select && create.is_materialized_view && create.to_table_id) { - StoragePtr to_table = DatabaseCatalog::instance().getTable({create.to_table_id.database_name, - create.to_table_id.table_name, - create.to_table_id.uuid}, - getContext()); - const auto & to_output_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - - ColumnsWithTypeAndName view_output_columns; - for (const auto & [name, type] : properties.columns.getAllPhysical()) - view_output_columns.emplace_back(type, name); - - Block input_columns = InterpreterSelectWithUnionQuery( + Block input_block = InterpreterSelectWithUnionQuery( create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); - ActionsDAG::makeConvertingActions( - input_columns.getColumnsWithTypeAndName(), - view_output_columns, - ActionsDAG::MatchColumnsMode::Name); + StoragePtr to_table = DatabaseCatalog::instance().getTable( + {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, + getContext() + ); + + Block to_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock(); + + ColumnsWithTypeAndName output_columns; + for (const auto & column : input_block) + output_columns.push_back(to_columns.findByName(column.name)->cloneEmpty()); ActionsDAG::makeConvertingActions( - view_output_columns, - to_output_columns.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); + input_block.getColumnsWithTypeAndName(), + output_columns, + ActionsDAG::MatchColumnsMode::Name + ); } DatabasePtr database; From 1b4cebcfb6f318d9ca98a9d3aae776a6ec2781ed Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 11 Mar 2022 10:04:35 +0000 Subject: [PATCH 022/110] Add new/delete overloads --- src/Common/Concepts.h | 14 +++++++ src/Common/memory.h | 77 +++++++++++++++++++++++++++++++-------- src/Common/new_delete.cpp | 54 ++++++++++++++++++++++++++- 3 files changed, 127 insertions(+), 18 deletions(-) create mode 100644 src/Common/Concepts.h diff --git a/src/Common/Concepts.h b/src/Common/Concepts.h new file mode 100644 index 00000000000..b1bf591024d --- /dev/null +++ b/src/Common/Concepts.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +namespace DB +{ + +template +concept OptionalArgument = requires(T &&...) +{ + requires(sizeof...(T) == 0 || sizeof...(T) == 1); +}; + +} diff --git a/src/Common/memory.h b/src/Common/memory.h index 41b10a57db4..3b819f295ca 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -1,8 +1,11 @@ #pragma once +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wold-style-cast" #include #include +#include #include #include @@ -14,13 +17,24 @@ # include #endif - namespace Memory { -inline ALWAYS_INLINE void * newImpl(std::size_t size) +inline ALWAYS_INLINE size_t alignToSizeT(std::align_val_t align) noexcept { - auto * ptr = malloc(size); + return static_cast(align); +} + +template ... TAlign> +requires DB::OptionalArgument +inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) +{ + void * ptr = nullptr; + if constexpr (sizeof...(TAlign) == 1) + ptr = aligned_alloc(alignToSizeT(align...), size); + else + ptr = malloc(size); + if (likely(ptr != nullptr)) return ptr; @@ -33,6 +47,11 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept return malloc(size); } +inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) noexcept +{ + return aligned_alloc(static_cast(align), size); +} + inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept { free(ptr); @@ -40,17 +59,24 @@ inline ALWAYS_INLINE void deleteImpl(void * ptr) noexcept #if USE_JEMALLOC -inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size) noexcept +template ... TAlign> +requires DB::OptionalArgument +inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size, TAlign... align) noexcept { if (unlikely(ptr == nullptr)) return; - sdallocx(ptr, size, 0); + if constexpr (sizeof...(TAlign) == 1) + sdallocx(ptr, size, MALLOCX_ALIGN(alignToSizeT(align...))); + else + sdallocx(ptr, size, 0); } #else -inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]]) noexcept +template ... TAlign> +requires DB::OptionalArgument +inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unused]], TAlign... /* align */) noexcept { free(ptr); } @@ -58,13 +84,14 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unuse #endif #if defined(OS_LINUX) -# include +# include #elif defined(OS_DARWIN) -# include +# include #endif - -inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) +template ... TAlign> +requires DB::OptionalArgument +inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size, TAlign... align) { size_t actual_size = size; @@ -72,26 +99,41 @@ inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size) /// The nallocx() function allocates no memory, but it performs the same size computation as the mallocx() function /// @note je_mallocx() != je_malloc(). It's expected they don't differ much in allocation logic. if (likely(size != 0)) - actual_size = nallocx(size, 0); + { + if constexpr (sizeof...(TAlign) == 1) + actual_size = nallocx(size, MALLOCX_ALIGN(alignToSizeT(align...))); + else + actual_size = nallocx(size, 0); + } #endif return actual_size; } -inline ALWAYS_INLINE void trackMemory(std::size_t size) +template ... TAlign> +requires DB::OptionalArgument +inline ALWAYS_INLINE void trackMemory(std::size_t size, TAlign... align) { - std::size_t actual_size = getActualAllocationSize(size); + std::size_t actual_size = getActualAllocationSize(size, align...); CurrentMemoryTracker::allocNoThrow(actual_size); } -inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0) noexcept +template ... TAlign> +requires DB::OptionalArgument +inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0, TAlign... align) noexcept { try { #if USE_JEMALLOC + /// @note It's also possible to use je_malloc_usable_size() here. if (likely(ptr != nullptr)) - CurrentMemoryTracker::free(sallocx(ptr, 0)); + { + if constexpr (sizeof...(TAlign) == 1) + CurrentMemoryTracker::free(sallocx(ptr, MALLOCX_ALIGN(alignToSizeT(align...)))); + else + CurrentMemoryTracker::free(sallocx(ptr, 0)); + } #else if (size) CurrentMemoryTracker::free(size); @@ -103,7 +145,10 @@ inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t #endif } catch (...) - {} + { + } } } + +#pragma GCC diagnostic pop diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index 8908d140b90..7b4bff04185 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,6 +1,7 @@ -#include -#include +#include #include +#include +#include #if defined(OS_DARWIN) && (USE_JEMALLOC) /// In case of OSX jemalloc register itself as a default zone allocator. @@ -53,12 +54,24 @@ void * operator new(std::size_t size) return Memory::newImpl(size); } +void * operator new(std::size_t size, std::align_val_t align) +{ + Memory::trackMemory(size, align); + return Memory::newImpl(size, align); +} + void * operator new[](std::size_t size) { Memory::trackMemory(size); return Memory::newImpl(size); } +void * operator new[](std::size_t size, std::align_val_t align) +{ + Memory::trackMemory(size, align); + return Memory::newImpl(size, align); +} + void * operator new(std::size_t size, const std::nothrow_t &) noexcept { Memory::trackMemory(size); @@ -71,6 +84,18 @@ void * operator new[](std::size_t size, const std::nothrow_t &) noexcept return Memory::newNoExept(size); } +void * operator new(std::size_t size, std::align_val_t align, const std::nothrow_t &) noexcept +{ + Memory::trackMemory(size, align); + return Memory::newNoExept(size, align); +} + +void * operator new[](std::size_t size, std::align_val_t align, const std::nothrow_t &) noexcept +{ + Memory::trackMemory(size, align); + return Memory::newNoExept(size, align); +} + /// delete /// C++17 std 21.6.2.1 (11) @@ -81,26 +106,51 @@ void * operator new[](std::size_t size, const std::nothrow_t &) noexcept /// It's unspecified whether size-aware or size-unaware version is called when deleting objects of /// incomplete type and arrays of non-class and trivially-destructible class types. + void operator delete(void * ptr) noexcept { Memory::untrackMemory(ptr); Memory::deleteImpl(ptr); } +void operator delete(void * ptr, std::align_val_t align) noexcept +{ + Memory::untrackMemory(ptr, 0, align); + Memory::deleteImpl(ptr); +} + void operator delete[](void * ptr) noexcept { Memory::untrackMemory(ptr); Memory::deleteImpl(ptr); } +void operator delete[](void * ptr, std::align_val_t align) noexcept +{ + Memory::untrackMemory(ptr, 0, align); + Memory::deleteImpl(ptr); +} + void operator delete(void * ptr, std::size_t size) noexcept { Memory::untrackMemory(ptr, size); Memory::deleteSized(ptr, size); } +void operator delete(void * ptr, std::size_t size, std::align_val_t align) noexcept +{ + Memory::untrackMemory(ptr, size, align); + Memory::deleteSized(ptr, size, align); +} + void operator delete[](void * ptr, std::size_t size) noexcept { Memory::untrackMemory(ptr, size); Memory::deleteSized(ptr, size); } + +void operator delete[](void * ptr, std::size_t size, std::align_val_t align) noexcept +{ + Memory::untrackMemory(ptr, size, align); + Memory::deleteSized(ptr, size, align); +} From 586853960530dad3dfe56ae07323b00cb679e811 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 12 Apr 2022 10:17:39 -0400 Subject: [PATCH 023/110] check only if TO table exists --- src/Interpreters/InterpreterCreateQuery.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 9a7f618696a..5ffbf8963b7 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1072,7 +1072,12 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create); /// Check type compatible for materialized dest table and select columns - if (create.select && create.is_materialized_view && create.to_table_id) + if (create.select && create.is_materialized_view && create.to_table_id && + DatabaseCatalog::instance().isTableExist( + {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, + getContext() + ) + ) { Block input_block = InterpreterSelectWithUnionQuery( create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); From a4f56f333052fa1ffa8d76d25087d98e5a54ed68 Mon Sep 17 00:00:00 2001 From: Julian Gilyadov Date: Mon, 4 Apr 2022 14:23:34 +0200 Subject: [PATCH 024/110] Throw exception when file cant be executed instead of displaying success --- src/Common/filesystemHelpers.cpp | 13 ++++++++++++- src/Common/filesystemHelpers.h | 2 ++ src/Dictionaries/ExecutableDictionarySource.cpp | 11 +++++++---- src/Dictionaries/ExecutablePoolDictionarySource.cpp | 10 +++++++--- .../UserDefinedExecutableFunctionFactory.cpp | 10 +++++++--- src/Storages/StorageExecutable.cpp | 12 ++++++++---- 6 files changed, 43 insertions(+), 15 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 6c5813ab29c..ca06b21ab3a 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -234,6 +234,11 @@ bool createFile(const std::string & path) DB::throwFromErrnoWithPath("Cannot create file: " + path, path, DB::ErrorCodes::CANNOT_CREATE_FILE); } +bool exists(const std::string & path) +{ + return faccessat(AT_FDCWD, path.c_str(), F_OK, AT_EACCESS) == 0; +} + bool canRead(const std::string & path) { struct stat st; @@ -249,7 +254,6 @@ bool canRead(const std::string & path) DB::throwFromErrnoWithPath("Cannot check read access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); } - bool canWrite(const std::string & path) { struct stat st; @@ -265,6 +269,13 @@ bool canWrite(const std::string & path) DB::throwFromErrnoWithPath("Cannot check write access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); } +bool canExecute(const std::string & path) +{ + if (exists(path)) + return faccessat(AT_FDCWD, path.c_str(), X_OK, AT_EACCESS) == 0; + DB::throwFromErrnoWithPath("Cannot check execute access to file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + time_t getModificationTime(const std::string & path) { struct stat st; diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 27041a2f0a8..b15073796a0 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -70,8 +70,10 @@ namespace FS { bool createFile(const std::string & path); +bool exists(const std::string & path); bool canRead(const std::string & path); bool canWrite(const std::string & path); +bool canExecute(const std::string & path); time_t getModificationTime(const std::string & path); Poco::Timestamp getModificationTimestamp(const std::string & path); diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 7a3550e7284..762e26cd2e2 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -15,12 +14,10 @@ #include #include -#include #include #include #include -#include namespace DB @@ -51,12 +48,18 @@ namespace command, user_scripts_path); - if (!std::filesystem::exists(std::filesystem::path(script_path))) + if (!FS::exists(script_path)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable file {} does not exist inside user scripts folder {}", command, user_scripts_path); + if (!FS::canExecute(script_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} is not executable inside user scripts folder {}", + command, + user_scripts_path); + command = std::move(script_path); } diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 62598c966e5..a7c152c02f0 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -20,7 +19,6 @@ #include #include - namespace DB { @@ -113,12 +111,18 @@ Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) command, user_scripts_path); - if (!std::filesystem::exists(std::filesystem::path(script_path))) + if (!FS::exists(script_path)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable file {} does not exist inside user scripts folder {}", command, user_scripts_path); + if (!FS::canExecute(script_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} is not executable inside user scripts folder {}", + command, + user_scripts_path); + command = std::move(script_path); } diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index d3a38f42e21..5b5c7911735 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -4,8 +4,6 @@ #include -#include - #include #include #include @@ -78,12 +76,18 @@ public: command, user_scripts_path); - if (!std::filesystem::exists(std::filesystem::path(script_path))) + if (!FS::exists(script_path)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable file {} does not exist inside user scripts folder {}", command, user_scripts_path); + if (!FS::canExecute(script_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} is not executable inside user scripts folder {}", + command, + user_scripts_path); + command = std::move(script_path); } diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index d9e97f98d56..577b4cab585 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -1,21 +1,19 @@ #include #include +#include #include -#include #include #include -#include #include #include #include #include -#include #include #include #include @@ -123,12 +121,18 @@ Pipe StorageExecutable::read( script_name, user_scripts_path); - if (!std::filesystem::exists(std::filesystem::path(script_path))) + if (!FS::exists(script_path)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable file {} does not exist inside user scripts folder {}", script_name, user_scripts_path); + if (!FS::canExecute(script_path)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} is not executable inside user scripts folder {}", + script_name, + user_scripts_path); + Pipes inputs; inputs.reserve(input_queries.size()); From 0679d4814dd37b7c942896336414ab9e3d2f3c4a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Apr 2022 18:58:40 +0000 Subject: [PATCH 025/110] allow to convert empty string to objects --- src/DataTypes/ObjectUtils.cpp | 5 ----- .../Serializations/SerializationObject.cpp | 7 +++++++ .../01825_type_json_empty_string.reference | 4 ++++ .../01825_type_json_empty_string.sql | 10 ++++++++++ .../01825_type_json_ephemeral.reference | 1 + .../0_stateless/01825_type_json_ephemeral.sql | 17 +++++++++++++++++ 6 files changed, 39 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01825_type_json_empty_string.reference create mode 100644 tests/queries/0_stateless/01825_type_json_empty_string.sql create mode 100644 tests/queries/0_stateless/01825_type_json_ephemeral.reference create mode 100644 tests/queries/0_stateless/01825_type_json_ephemeral.sql diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index cbabc71a965..629cd9c28b2 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -139,11 +139,6 @@ void convertObjectsToTuples(Block & block, const NamesAndTypesList & extended_st if (!isObject(column.type)) continue; - if (!isObject(column.type)) - throw Exception(ErrorCodes::TYPE_MISMATCH, - "Type for column '{}' mismatch in columns list and in block. In list: {}, in block: {}", - column.name, column.type->getName(), column.type->getName()); - const auto & column_object = assert_cast(*column.column); const auto & subcolumns = column_object.getSubcolumns(); diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 64d79d107e1..82f9552fb86 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -136,10 +136,17 @@ void SerializationObject::deserializeTextImpl(IColumn & column, Reader & reader(buf); std::optional result; + /// Treat empty string as an empty object + /// for better CAST from String to Object. + if (!buf.empty()) { auto parser = parsers_pool.get([] { return new Parser; }); result = parser->parse(buf.data(), buf.size()); } + else + { + result = ParseResult{}; + } if (!result) throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot parse object"); diff --git a/tests/queries/0_stateless/01825_type_json_empty_string.reference b/tests/queries/0_stateless/01825_type_json_empty_string.reference new file mode 100644 index 00000000000..7e3d2e54336 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_empty_string.reference @@ -0,0 +1,4 @@ +1 (0,'') +2 (1,'v1') +3 (0,'') +4 (2,'') diff --git a/tests/queries/0_stateless/01825_type_json_empty_string.sql b/tests/queries/0_stateless/01825_type_json_empty_string.sql new file mode 100644 index 00000000000..5af3af51208 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_empty_string.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS t_json_empty_str; +SET allow_experimental_object_type = 1; + +CREATE TABLE t_json_empty_str(id UInt32, o JSON) ENGINE = Memory; + +INSERT INTO t_json_empty_str VALUES (1, ''), (2, '{"k1": 1, "k2": "v1"}'), (3, '{}'), (4, '{"k1": 2}'); + +SELECT * FROM t_json_empty_str ORDER BY id; + +DROP TABLE t_json_empty_str; diff --git a/tests/queries/0_stateless/01825_type_json_ephemeral.reference b/tests/queries/0_stateless/01825_type_json_ephemeral.reference new file mode 100644 index 00000000000..67d2a24d3fe --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_ephemeral.reference @@ -0,0 +1 @@ +PushEvent some-repo (('https://avatars.githubusercontent.com/u/123213213?','github-actions','',123123123,'github-actions[bot]','https://api.github.com/users/github-actions[bot]'),'2022-01-04 07:00:00',(1001001010101,'some-repo','https://api.github.com/repos/some-repo'),'PushEvent') diff --git a/tests/queries/0_stateless/01825_type_json_ephemeral.sql b/tests/queries/0_stateless/01825_type_json_ephemeral.sql new file mode 100644 index 00000000000..276fdd2380a --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_ephemeral.sql @@ -0,0 +1,17 @@ +SET allow_experimental_object_type = 1; + +DROP TABLE IF EXISTS t_github_json; + +CREATE table t_github_json +( + event_type LowCardinality(String) DEFAULT JSONExtractString(message_raw, 'type'), + repo_name LowCardinality(String) DEFAULT JSONExtractString(message_raw, 'repo', 'name'), + message JSON DEFAULT message_raw, + message_raw String EPHEMERAL +) ENGINE = MergeTree ORDER BY (event_type, repo_name); + +INSERT INTO t_github_json (message_raw) FORMAT JSONEachRow {"message_raw": "{\"type\":\"PushEvent\", \"created_at\": \"2022-01-04 07:00:00\", \"actor\":{\"avatar_url\":\"https://avatars.githubusercontent.com/u/123213213?\",\"display_login\":\"github-actions\",\"gravatar_id\":\"\",\"id\":123123123,\"login\":\"github-actions[bot]\",\"url\":\"https://api.github.com/users/github-actions[bot]\"},\"repo\":{\"id\":1001001010101,\"name\":\"some-repo\",\"url\":\"https://api.github.com/repos/some-repo\"}}"} + +SELECT * FROM t_github_json ORDER BY event_type, repo_name; + +DROP TABLE t_github_json; From 3d52afed71a51a29c484d9c422eca85b46f30e35 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 12 Apr 2022 15:10:11 -0400 Subject: [PATCH 026/110] only check columns sets intersection, some refactoring --- src/Interpreters/InterpreterCreateQuery.cpp | 43 +++++++++++---------- 1 file changed, 23 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5ffbf8963b7..8e13ce1d487 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1072,32 +1072,35 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create); /// Check type compatible for materialized dest table and select columns - if (create.select && create.is_materialized_view && create.to_table_id && - DatabaseCatalog::instance().isTableExist( - {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, - getContext() - ) - ) + if (create.select && create.is_materialized_view && create.to_table_id) { - Block input_block = InterpreterSelectWithUnionQuery( - create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); - - StoragePtr to_table = DatabaseCatalog::instance().getTable( + if (StoragePtr to_table = DatabaseCatalog::instance().tryGetTable( {create.to_table_id.database_name, create.to_table_id.table_name, create.to_table_id.uuid}, getContext() - ); + )) + { + Block input_block = InterpreterSelectWithUnionQuery( + create.select->clone(), getContext(), SelectQueryOptions().analyze()).getSampleBlock(); - Block to_columns = to_table->getInMemoryMetadataPtr()->getSampleBlock(); + Block output_block = to_table->getInMemoryMetadataPtr()->getSampleBlock(); - ColumnsWithTypeAndName output_columns; - for (const auto & column : input_block) - output_columns.push_back(to_columns.findByName(column.name)->cloneEmpty()); + ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName output_columns; + for (const auto & input_column : input_block) + { + if (const auto * output_column = output_block.findByName(input_column.name)) + { + input_columns.push_back(input_column.cloneEmpty()); + output_columns.push_back(output_column->cloneEmpty()); + } + } - ActionsDAG::makeConvertingActions( - input_block.getColumnsWithTypeAndName(), - output_columns, - ActionsDAG::MatchColumnsMode::Name - ); + ActionsDAG::makeConvertingActions( + input_columns, + output_columns, + ActionsDAG::MatchColumnsMode::Position + ); + } } DatabasePtr database; From c675e2187292fb06a55c455040338f76b2793404 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 12 Apr 2022 15:49:08 -0400 Subject: [PATCH 027/110] tests updated --- .../0_stateless/01880_materialized_view_to_table_type_check.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql b/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql index 342ef08bc89..2da9884ba8e 100644 --- a/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql +++ b/tests/queries/0_stateless/01880_materialized_view_to_table_type_check.sql @@ -8,8 +8,6 @@ CREATE TABLE test(`id` Int32, `pv` AggregateFunction(sum, Int32)) ENGINE = Aggre CREATE MATERIALIZED VIEW test_mv to test(`id` Int32, `pv` AggregateFunction(sum, Int32)) as SELECT id, sumState(1) as pv from test_input group by id; -- { serverError 70 } -DROP VIEW test_mv; - INSERT INTO test_input SELECT toInt32(number % 1000) AS id FROM numbers(10); select '----------test--------:'; select * from test; From b066585cae14bc7f99a64653b8555efc232ef72d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Apr 2022 22:12:58 +0200 Subject: [PATCH 028/110] Apply suggestions from code review Co-authored-by: Vladimir C --- tests/queries/0_stateless/01825_type_json_empty_string.sql | 2 ++ tests/queries/0_stateless/01825_type_json_ephemeral.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01825_type_json_empty_string.sql b/tests/queries/0_stateless/01825_type_json_empty_string.sql index 5af3af51208..b3f2a7cb120 100644 --- a/tests/queries/0_stateless/01825_type_json_empty_string.sql +++ b/tests/queries/0_stateless/01825_type_json_empty_string.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + DROP TABLE IF EXISTS t_json_empty_str; SET allow_experimental_object_type = 1; diff --git a/tests/queries/0_stateless/01825_type_json_ephemeral.sql b/tests/queries/0_stateless/01825_type_json_ephemeral.sql index 276fdd2380a..4485510e419 100644 --- a/tests/queries/0_stateless/01825_type_json_ephemeral.sql +++ b/tests/queries/0_stateless/01825_type_json_ephemeral.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + SET allow_experimental_object_type = 1; DROP TABLE IF EXISTS t_github_json; From 835f752a1ae9135ac35ff7da98a6cba0d34c4298 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 13 Apr 2022 06:19:59 +0000 Subject: [PATCH 029/110] Add maybe_unused for align arguments --- src/Common/memory.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/memory.h b/src/Common/memory.h index 3b819f295ca..8a6b98933fb 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -91,7 +91,7 @@ inline ALWAYS_INLINE void deleteSized(void * ptr, std::size_t size [[maybe_unuse template ... TAlign> requires DB::OptionalArgument -inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size, TAlign... align) +inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size, TAlign... align [[maybe_unused]]) { size_t actual_size = size; @@ -120,7 +120,7 @@ inline ALWAYS_INLINE void trackMemory(std::size_t size, TAlign... align) template ... TAlign> requires DB::OptionalArgument -inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0, TAlign... align) noexcept +inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0, TAlign... align [[maybe_unused]]) noexcept { try { From 1b267462d836d92a5b84449accf8a1781165d42f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Apr 2022 11:27:47 +0200 Subject: [PATCH 030/110] Update CachedReadBufferFromRemoteFS.cpp --- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index d39d54970dc..1cc819a7443 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -336,7 +336,7 @@ SeekableReadBufferPtr CachedReadBufferFromRemoteFS::getImplementationBuffer(File size_t seek_offset = file_offset_of_buffer_end - range.left; if (file_offset_of_buffer_end < range.left) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} > {}", file_offset_of_buffer_end, range.left); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed invariant. Expected {} > {}", file_offset_of_buffer_end, range.left); read_buffer_for_file_segment->seek(seek_offset, SEEK_SET); From a611566266a33a2cd593aafc8f3c7bd559c5eabb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Apr 2022 11:33:15 +0200 Subject: [PATCH 031/110] Update FileCache.h --- src/Common/FileCache.h | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index b1d46569949..01b639ea1c9 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -72,6 +72,15 @@ public: */ virtual FileSegmentsHolder getOrSet(const Key & key, size_t offset, size_t size) = 0; + /** + * Segments in returned list are ordered in ascending order and represent a full contiguous + * interval (no holes). Each segment in returned list has state: DOWNLOADED, DOWNLOADING or EMPTY. + * + * If file segment has state EMPTY, then it is also marked as "detached". E.g. it is "detached" + * from cache (not owned by cache), and as a result will never change it's state and will be destructed + * with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change + * it's state (and become DOWNLOADED). + */ virtual FileSegmentsHolder get(const Key & key, size_t offset, size_t size) = 0; virtual FileSegmentsHolder setDownloading(const Key & key, size_t offset, size_t size) = 0; From 053892777f2202d604a1d201ea0fcb97f0713489 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Apr 2022 11:35:12 +0200 Subject: [PATCH 032/110] Update FileSegment.cpp --- src/Common/FileSegment.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 92a478f3476..4f144362ed9 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -667,6 +667,9 @@ FileSegmentsHolder::~FileSegmentsHolder() if (file_segment->detached) { + /// This file segment is not owned by cache, so it will be destructed + /// at this point, therefore no completion required. + assert(file_segment->state() == State::EMPTY); file_segment_it = file_segments.erase(current_file_segment_it); continue; } From 44ecfd925bf31a24fec382baa4d46e0c94489b15 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Apr 2022 11:35:46 +0200 Subject: [PATCH 033/110] Update FileSegment.cpp --- src/Common/FileSegment.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 4f144362ed9..50a1ecb4a6c 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -669,7 +669,7 @@ FileSegmentsHolder::~FileSegmentsHolder() { /// This file segment is not owned by cache, so it will be destructed /// at this point, therefore no completion required. - assert(file_segment->state() == State::EMPTY); + assert(file_segment->state() == FileSegment::State::EMPTY); file_segment_it = file_segments.erase(current_file_segment_it); continue; } From 7b0f1fbb889bff72a7e5013d967d7ac7053c49b4 Mon Sep 17 00:00:00 2001 From: Michael Lex Date: Wed, 13 Apr 2022 12:05:51 +0200 Subject: [PATCH 034/110] Extend TLS documentation. Information about used defaults. Better explanation of the extendedVerification option (which enables hostname verification) Better explanation of the caConfig option and how files vs. paths are treated. Mention TLS config in CLI documentation and link to server-side settings (which are the same). --- docs/en/interfaces/cli.md | 9 ++- .../settings.md | 56 +++++++++---------- 2 files changed, 35 insertions(+), 30 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index eaf7a96ce42..a252f55de2c 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -124,7 +124,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. -- `--secure` – If specified, will connect to server over secure connection. +- `--secure` – If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl). - `--history_file` — Path to a file containing command history. - `--param_` — Value for a [query with parameters](#cli-queries-with-parameters). - `--hardware-utilization` — Print hardware utilization information in progress bar. @@ -148,7 +148,12 @@ Example of a config file: username password - False + true + + + /etc/ssl/cert.pem + + ``` diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 301b348925f..fb53fd38fe3 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -366,12 +366,12 @@ Opens `https://tabix.io/` when accessing `http://localhost: http_port`.
]]>
-``` +``` ## hsts_max_age {#hsts-max-age} - -Expired time for HSTS in seconds. The default value is 0 means clickhouse disabled HSTS. If you set a positive number, the HSTS will be enabled and the max-age is the number you set. - -**Example** + +Expired time for HSTS in seconds. The default value is 0 means clickhouse disabled HSTS. If you set a positive number, the HSTS will be enabled and the max-age is the number you set. + +**Example** ```xml 600000 @@ -468,7 +468,7 @@ To enable authentication, set `interserver_http_credentials.allow_empty` to `tru After configuring all replicas set `allow_empty` to `false` or remove this setting. It makes authentication with new credentials mandatory. -To change existing credentials, move the username and the password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this point the server uses new credentials to connect to other replicas and accepts connections with either new or old credentials. +To change existing credentials, move the username and the password to `interserver_http_credentials.old` section and update `user` and `password` with new values. At this point the server uses new credentials to connect to other replicas and accepts connections with either new or old credentials. ``` xml @@ -834,7 +834,7 @@ The value 0 means that you can delete all tables without any restrictions. ClickHouse uses threads from the Global Thread pool to process queries. If there is no idle thread to process a query, then a new thread is created in the pool. `max_thread_pool_size` limits the maximum number of threads in the pool. -Possible values: +Possible values: - Positive integer. @@ -850,7 +850,7 @@ Default value: `10000`. If the number of **idle** threads in the Global Thread pool is greater than `max_thread_pool_free_size`, then ClickHouse releases resources occupied by some threads and the pool size is decreased. Threads can be created again if necessary. -Possible values: +Possible values: - Positive integer. @@ -866,7 +866,7 @@ Default value: `1000`. The maximum number of jobs that can be scheduled on the Global Thread pool. Increasing queue size leads to larger memory usage. It is recommended to keep this value equal to [max_thread_pool_size](#max-thread-pool-size). -Possible values: +Possible values: - Positive integer. @@ -941,30 +941,30 @@ For more information, see the MergeTreeSettings.h header file. SSL client/server configuration. -Support for SSL is provided by the `libpoco` library. The interface is described in the file [SSLManager.h](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h) +Support for SSL is provided by the `libpoco` library. The available configuration options are explained in [SSLManager.h](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h). Default values can be found in [SSLManager.cpp](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/src/SSLManager.cpp). Keys for server/client settings: - privateKeyFile – The path to the file with the secret key of the PEM certificate. The file may contain a key and certificate at the same time. - certificateFile – The path to the client/server certificate file in PEM format. You can omit it if `privateKeyFile` contains the certificate. -- caConfig – The path to the file or directory that contains trusted root certificates. -- verificationMode – The method for checking the node’s certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`. -- verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. -- loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. \| -- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. -- cacheSessions – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`. -- sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`. -- sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. -- sessionTimeout – Time for caching the session on the server. -- extendedVerification – Automatically extended verification of certificates after the session ends. Acceptable values: `true`, `false`. -- requireTLSv1 – Require a TLSv1 connection. Acceptable values: `true`, `false`. -- requireTLSv1_1 – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. -- requireTLSv1_2 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. -- fips – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. -- privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` ConsoleCertificateHandler ` . -- disableProtocols – Protocols that are not allowed to use. -- preferServerCiphers – Preferred server ciphers on the client. +- caConfig (default: none) – The path to the file or directory that contains trusted CA certificates. If this points to a file, it must be in PEM format and can contain several CA certificates. If this points to a directory, it must contain one .pem file per CA certificate. The filenames are looked up by the CA subject name hash value. Details can be found in the man page of [SSL_CTX_load_verify_locations](https://www.openssl.org/docs/man3.0/man3/SSL_CTX_load_verify_locations.html). +- verificationMode (default: relaxed) – The method for checking the node’s certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`. +- verificationDepth (default: 9) – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. +- loadDefaultCAFile (default: true) – Wether built-in CA certificates for OpenSSL will be used. ClickHouse assumes that builtin CA certificates are in the file `/etc/ssl/cert.pem` (resp. the directory `/etc/ssl/certs`) or in file (resp. directory) specified by the environment variable `SSL_CERT_FILE` (resp. `SSL_CERT_DIR`). +- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`) - Supported OpenSSL encryptions. +- cacheSessions (default: false) – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`. +- sessionIdContext (default: `${application.name}`) – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`. +- sessionCacheSize (default: [1024\*20](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1978)) – The maximum number of sessions that the server caches. A value of 0 means unlimited sessions. +- sessionTimeout (default: [2h](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1926)) – Time for caching the session on the server. +- extendedVerification (default: false) – If enabled, verify that the certificate CN or SAN matches the peer hostname. +- requireTLSv1 (default: false) – Require a TLSv1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_1 (default: false) – Require a TLSv1.1 connection. Acceptable values: `true`, `false`. +- requireTLSv1_2 (default: false) – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. +- fips (default: false) – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. +- privateKeyPassphraseHandler (default: `KeyConsoleHandler`)– Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. +- invalidCertificateHandler (default: `ConsoleCertificateHandler`) – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` ConsoleCertificateHandler ` . +- disableProtocols (default: "") – Protocols that are not allowed to use. +- preferServerCiphers (default: false) – Preferred server ciphers on the client. **Example of settings:** From 07cfa980ace109ed333a9185bd7a0a63f73e8903 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Apr 2022 12:08:25 +0200 Subject: [PATCH 035/110] Update FileCache.h --- src/Common/FileCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/FileCache.h b/src/Common/FileCache.h index 01b639ea1c9..983156959de 100644 --- a/src/Common/FileCache.h +++ b/src/Common/FileCache.h @@ -78,7 +78,7 @@ public: * * If file segment has state EMPTY, then it is also marked as "detached". E.g. it is "detached" * from cache (not owned by cache), and as a result will never change it's state and will be destructed - * with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change + * with the destruction of the holder, while in getOrSet() EMPTY file segments can eventually change * it's state (and become DOWNLOADED). */ virtual FileSegmentsHolder get(const Key & key, size_t offset, size_t size) = 0; From a4d74f975e8ae91513a87c1427b4d8d32f0ba379 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 13 Apr 2022 10:20:29 +0000 Subject: [PATCH 036/110] Use atomic instead of mutex + condvar in ParallelReadBuffer --- src/IO/ParallelReadBuffer.cpp | 16 ++++++++-------- src/IO/ParallelReadBuffer.h | 4 +--- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 79a16347094..fac6c14c082 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -205,12 +205,8 @@ bool ParallelReadBuffer::nextImpl() void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) { SCOPE_EXIT({ - std::lock_guard lock{mutex}; - --active_working_reader; - if (active_working_reader == 0) - { - readers_done.notify_all(); - } + if (--active_working_reader == 0) + active_working_reader.notify_all(); }); try @@ -265,8 +261,12 @@ void ParallelReadBuffer::finishAndWait() { emergency_stop = true; - std::unique_lock lock{mutex}; - readers_done.wait(lock, [&] { return active_working_reader == 0; }); + size_t active_readers = active_working_reader.load(); + while (active_readers != 0) + { + active_working_reader.wait(active_readers); + active_readers = active_working_reader.load(); + } } } diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index eda88969afa..50bec6663ce 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -135,9 +135,7 @@ private: Segment current_segment; size_t max_working_readers; - size_t active_working_reader{0}; - // Triggered when all reader workers are done - std::condition_variable readers_done; + std::atomic_size_t active_working_reader{0}; CallbackRunner schedule; From 7af954cef9428afb68e7de7584ae2e8dd4baa6fd Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 13 Apr 2022 14:37:44 +0300 Subject: [PATCH 037/110] Update clickhouse-test --- tests/clickhouse-test | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b52a9f4f026..5f15ecf3dc6 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1176,6 +1176,9 @@ def check_server_started(args): except TimeoutError: print("\nConnection timeout, will not retry") break + except Exception as e: + print("\nUexpected exception, will not retry: ", str(e)) + break print('\nAll connection tries failed') sys.stdout.flush() From b2905c5aed8575306e224fba4a834f7239be2811 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 13 Apr 2022 13:50:11 +0200 Subject: [PATCH 038/110] Update src/IO/ParallelReadBuffer.cpp Co-authored-by: tavplubix --- src/IO/ParallelReadBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index fac6c14c082..3db4a45778e 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -205,7 +205,7 @@ bool ParallelReadBuffer::nextImpl() void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) { SCOPE_EXIT({ - if (--active_working_reader == 0) + if (active_working_reader.fetch_sub(1) == 1) active_working_reader.notify_all(); }); From ab80b94f4573bff6d90e55811458d049be964e50 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Apr 2022 15:18:49 +0300 Subject: [PATCH 039/110] Update ThreadPool.cpp --- src/Common/ThreadPool.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 15ae64f17fb..c5f806c8a89 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -245,7 +245,7 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ while (true) { /// This is inside the loop to also reset previous thread names set inside the jobs. - setThreadName("ThreadPoolIdle"); + setThreadName("ThreadPool"); Job job; bool need_shutdown = false; From 9ae047df1d7b35dce0ba8ca7af0c90ba029430a0 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 13 Apr 2022 14:30:10 +0200 Subject: [PATCH 040/110] Update CachedReadBufferFromRemoteFS.cpp --- src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 1cc819a7443..90a1d4ad43e 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -336,7 +336,7 @@ SeekableReadBufferPtr CachedReadBufferFromRemoteFS::getImplementationBuffer(File size_t seek_offset = file_offset_of_buffer_end - range.left; if (file_offset_of_buffer_end < range.left) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed invariant. Expected {} > {}", file_offset_of_buffer_end, range.left); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invariant failed. Expected {} > {} (current offset > file segment's start offset)", file_offset_of_buffer_end, range.left); read_buffer_for_file_segment->seek(seek_offset, SEEK_SET); From 348cae0d165d00a467844779e5a2db952e440013 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 13 Apr 2022 12:34:40 +0000 Subject: [PATCH 041/110] Fix possible segfault in schema inference for JSON formats --- src/Formats/JSONEachRowUtils.cpp | 7 +++++++ ...268_json_wrong_root_type_in_schema_inference.reference | 0 .../02268_json_wrong_root_type_in_schema_inference.sql | 8 ++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.reference create mode 100644 tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index 5e1830655f9..3d0634e0259 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -255,6 +255,10 @@ struct JSONEachRowFieldsExtractor std::vector extract(const Element & element) { /// {..., "" : , ...} + + if (!element.isObject()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Root JSON value is not an object"); + auto object = element.getObject(); std::vector fields; fields.reserve(object.size()); @@ -287,6 +291,9 @@ struct JSONCompactEachRowFieldsExtractor std::vector extract(const Element & element) { /// [..., , ...] + if (!element.isArray()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Root JSON value is not an array"); + auto array = element.getArray(); std::vector fields; fields.reserve(array.size()); diff --git a/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.reference b/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql b/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql new file mode 100644 index 00000000000..2e66635a752 --- /dev/null +++ b/tests/queries/0_stateless/02268_json_wrong_root_type_in_schema_inference.sql @@ -0,0 +1,8 @@ +-- Tags: no-backward-compatibility-check:22.4.1.1 + +insert into function file('02268_data.jsonl', 'TSV') select 1; +select * from file('02268_data.jsonl'); --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE} + +insert into function file('02268_data.jsonCompactEachRow', 'TSV') select 1; +select * from file('02268_data.jsonCompactEachRow'); --{serverError CANNOT_EXTRACT_TABLE_STRUCTURE} + From 6dbc138e76bb7a75aa2085d7ca69235264e768b6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Apr 2022 11:05:40 +0300 Subject: [PATCH 042/110] Remove duplicated parts removal from mutation. This commit reverts 701e2ffd795814fbd0083d9b24de236774d224cd ("Fix possible mutation stuck due to race with DROP_RANGE"), and also adds some clarification comments. Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c5798aaefe5..c368aa3303d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -357,9 +357,13 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( current_parts.remove(*drop_range_part_name); } + /// During inserting to queue (insertUnlocked()) we remove part for + /// DROP_RANGE only for DROP PART but not for DROP PARTITION. virtual_parts.remove(*drop_range_part_name); - removeCoveredPartsFromMutations(*drop_range_part_name, /*remove_part = */ true, /*remove_covered_parts = */ false); + /// NOTE: we don't need to remove part/covered parts from mutations (removeCoveredPartsFromMutations()) here because: + /// - for DROP PART we have this during inserting to queue (see insertUnlocked()) + /// - for DROP PARTITION we have this in the loop above (when we adding parts to current_parts) } if (entry->type == LogEntry::DROP_RANGE) From dc460f76d8e75d6f0f3c5a8bc02db7304b784db7 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 13 Apr 2022 15:58:15 +0000 Subject: [PATCH 043/110] Print progress bar on each ProfileEvents packet --- src/Client/ClientBase.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 929f0a48e2b..c9b7bb3a8ee 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -954,6 +954,9 @@ void ClientBase::onProfileEvents(Block & block) auto elapsed_time = profile_events.watch.elapsedMicroseconds(); progress_indication.updateThreadEventData(thread_times, elapsed_time); + if (need_render_progress) + progress_indication.writeProgress(); + if (profile_events.print) { if (profile_events.watch.elapsedMilliseconds() >= profile_events.delay_ms) From 66fdf35dfdbd5d2c67545c08ba44eb46525213c6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 13 Apr 2022 18:01:22 +0200 Subject: [PATCH 044/110] remove outdated parts immediately on drop partition --- src/Storages/MergeTree/MergeTreeData.cpp | 21 +++++++++++++++++---- src/Storages/MergeTree/MergeTreeData.h | 4 +++- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 4 files changed, 26 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e1e698d41e3..29b3083c38f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2878,8 +2878,7 @@ void MergeTreeData::removePartsFromWorkingSet( } MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( - MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, - bool clear_without_timeout, DataPartsLock & lock) + MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock) { DataPartsVector parts_to_remove; @@ -2947,6 +2946,13 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet( parts_to_remove.emplace_back(part); } + bool clear_without_timeout = true; + /// We a going to remove active parts covered by drop_range without timeout. + /// Let's also reset timeout for inactive parts. + auto inactive_parts_to_remove_immediately = getDataPartsVectorInPartitionForInternalUsage(DataPartState::Outdated, drop_range.partition_id, &lock); + for (auto & part : inactive_parts_to_remove_immediately) + part->remove_time.store(0, std::memory_order_relaxed); + /// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice removePartsFromWorkingSet(txn, parts_to_remove, clear_without_timeout, lock); @@ -3382,7 +3388,8 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartiti return getVisibleDataPartsVectorInPartition(local_context->getCurrentTransaction().get(), partition_id); } -MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartition(MergeTreeTransaction * txn, const String & partition_id, DataPartsLock * acquired_lock) const +MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartition( + MergeTreeTransaction * txn, const String & partition_id, DataPartsLock * acquired_lock) const { if (txn) { @@ -3398,7 +3405,13 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartiti return res; } - DataPartStateAndPartitionID state_with_partition{MergeTreeDataPartState::Active, partition_id}; + return getDataPartsVectorInPartitionForInternalUsage(MergeTreeDataPartState::Active, partition_id, acquired_lock); +} + +MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVectorInPartitionForInternalUsage( + const MergeTreeData::DataPartState & state, const String & partition_id, DataPartsLock * acquired_lock) const +{ + DataPartStateAndPartitionID state_with_partition{state, partition_id}; auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); return DataPartsVector( diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 3a099c471c2..1601a6714d3 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -494,6 +494,8 @@ public: DataPartsVector getVisibleDataPartsVectorInPartition(ContextPtr local_context, const String & partition_id) const; DataPartsVector getVisibleDataPartsVectorInPartitions(ContextPtr local_context, const std::unordered_set & partition_ids) const; + DataPartsVector getDataPartsVectorInPartitionForInternalUsage(const DataPartState & state, const String & partition_id, DataPartsLock * acquired_lock = nullptr) const; + /// Returns the part with the given name and state or nullptr if no such part. DataPartPtr getPartIfExists(const String & part_name, const DataPartStates & valid_states); DataPartPtr getPartIfExists(const MergeTreePartInfo & part_info, const DataPartStates & valid_states); @@ -577,7 +579,7 @@ public: /// Removes all parts from the working set parts /// for which (partition_id = drop_range.partition_id && min_block >= drop_range.min_block && max_block <= drop_range.max_block). /// Used in REPLACE PARTITION command; - DataPartsVector removePartsInRangeFromWorkingSet(MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, bool clear_without_timeout, + DataPartsVector removePartsInRangeFromWorkingSet(MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock); /// Restores Outdated part and adds it to working set diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b25b47ac772..e322d8785fa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1618,7 +1618,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con /// If it is REPLACE (not ATTACH), remove all parts which max_block_number less then min_block_number of the first new block if (replace) - removePartsInRangeFromWorkingSet(local_context->getCurrentTransaction().get(), drop_range, true, data_parts_lock); + removePartsInRangeFromWorkingSet(local_context->getCurrentTransaction().get(), drop_range, data_parts_lock); } PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 0e61d8b6f81..243637d910b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1886,7 +1886,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) DataPartsVector parts_to_remove; { auto data_parts_lock = lockParts(); - parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range_info, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range_info, data_parts_lock); if (parts_to_remove.empty()) { if (!drop_range_info.isFakeDropRangePart()) @@ -2019,7 +2019,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (parts_to_add.empty() && replace) { - parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, data_parts_lock); String parts_to_remove_str; for (const auto & part : parts_to_remove) { @@ -2257,7 +2257,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) transaction.commit(&data_parts_lock); if (replace) { - parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, data_parts_lock); String parts_to_remove_str; for (const auto & part : parts_to_remove) { @@ -6487,7 +6487,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( auto data_parts_lock = lockParts(); transaction.commit(&data_parts_lock); if (replace) - parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, data_parts_lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, data_parts_lock); } PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); @@ -6694,7 +6694,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta else zkutil::KeeperMultiException::check(code, ops, op_results); - parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, true, lock); + parts_to_remove = removePartsInRangeFromWorkingSet(NO_TRANSACTION_RAW, drop_range, lock); transaction.commit(&lock); } From 42726639f34567b22e24ae4bafb506c1ef3b808c Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 13 Apr 2022 19:27:38 +0000 Subject: [PATCH 045/110] Check ORC/Parquet/Arrow format magic bytes before loading file in memory --- .../Formats/Impl/ArrowBlockInputFormat.cpp | 2 +- .../Formats/Impl/ArrowBufferedStreams.cpp | 26 +++++++++++++++++-- .../Formats/Impl/ArrowBufferedStreams.h | 11 +++++++- .../Formats/Impl/ORCBlockInputFormat.cpp | 2 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- src/Storages/Hive/HiveFile.cpp | 4 +-- tests/integration/test_storage_s3/test.py | 15 +++++++++++ 7 files changed, 54 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 792ebd09392..07331d82bb8 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -111,7 +111,7 @@ static std::shared_ptr createStreamReader(ReadBuffer & static std::shared_ptr createFileReader(ReadBuffer & in, const FormatSettings & format_settings, std::atomic & is_stopped) { - auto arrow_file = asArrowFile(in, format_settings, is_stopped); + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Arrow", ARROW_MAGIC_BYTES); if (is_stopped) return nullptr; diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index 484a3a17f8f..8573a560d02 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +23,7 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_FILE_SIZE; + extern const int INCORRECT_DATA; } ArrowBufferedOutputStream::ArrowBufferedOutputStream(WriteBuffer & out_) : out{out_}, is_open{true} @@ -139,7 +141,12 @@ arrow::Status ArrowInputStreamFromReadBuffer::Close() return arrow::Status(); } -std::shared_ptr asArrowFile(ReadBuffer & in, const FormatSettings & settings, std::atomic & is_cancelled) +std::shared_ptr asArrowFile( + ReadBuffer & in, + const FormatSettings & settings, + std::atomic & is_cancelled, + const std::string & format_name, + const std::string & magic_bytes) { if (auto * fd_in = dynamic_cast(&in)) { @@ -158,8 +165,23 @@ std::shared_ptr asArrowFile(ReadBuffer & in, const // fallback to loading the entire file in memory std::string file_data; { + PeekableReadBuffer buf(in); + std::string magic_bytes_from_data; + magic_bytes_from_data.resize(magic_bytes.size()); + bool read_magic_bytes = false; + try + { + PeekableReadBufferCheckpoint checkpoint(buf, true); + buf.readStrict(magic_bytes_from_data.data(), magic_bytes_from_data.size()); + read_magic_bytes = true; + } + catch (const Exception &) {} + + if (!read_magic_bytes || magic_bytes_from_data != magic_bytes) + throw Exception(ErrorCodes::INCORRECT_DATA, "Not a {} file", format_name); + WriteBufferFromString file_buffer(file_data); - copyData(in, file_buffer, is_cancelled); + copyData(buf, file_buffer, is_cancelled); } return std::make_shared(arrow::Buffer::FromString(std::move(file_data))); diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index e06eab04f1b..4ad0ecdf012 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -6,6 +6,10 @@ #include #include +#define ORC_MAGIC_BYTES "ORC" +#define PARQUET_MAGIC_BYTES "PAR1" +#define ARROW_MAGIC_BYTES "ARROW1" + namespace DB { @@ -86,7 +90,12 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(ArrowInputStreamFromReadBuffer); }; -std::shared_ptr asArrowFile(ReadBuffer & in, const FormatSettings & settings, std::atomic & is_cancelled); +std::shared_ptr asArrowFile( + ReadBuffer & in, + const FormatSettings & settings, + std::atomic & is_cancelled, + const std::string & format_name, + const std::string & magic_bytes); } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 333129aee81..1531c0d2794 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -116,7 +116,7 @@ static void getFileReaderAndSchema( const FormatSettings & format_settings, std::atomic & is_stopped) { - auto arrow_file = asArrowFile(in, format_settings, is_stopped); + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "ORC", ORC_MAGIC_BYTES); if (is_stopped) return; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index af16d30bcfe..86987c665e0 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -117,7 +117,7 @@ static void getFileReaderAndSchema( const FormatSettings & format_settings, std::atomic & is_stopped) { - auto arrow_file = asArrowFile(in, format_settings, is_stopped); + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES); if (is_stopped) return; THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(std::move(arrow_file), arrow::default_memory_pool(), &file_reader)); diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 02c92770274..57acbdd577b 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -150,7 +150,7 @@ void HiveORCFile::prepareReader() in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); std::atomic is_stopped{0}; - auto result = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool()); + auto result = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings, is_stopped, "ORC", ORC_MAGIC_BYTES), arrow::default_memory_pool()); THROW_ARROW_NOT_OK(result.status()); reader = std::move(result).ValueOrDie(); } @@ -270,7 +270,7 @@ void HiveParquetFile::prepareReader() in = std::make_unique(namenode_url, path, getContext()->getGlobalContext()->getConfigRef()); auto format_settings = getFormatSettings(getContext()); std::atomic is_stopped{0}; - THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped), arrow::default_memory_pool(), &reader)); + THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(*in, format_settings, is_stopped, "Parquet", PARQUET_MAGIC_BYTES), arrow::default_memory_pool(), &reader)); } void HiveParquetFile::loadSplitMinMaxIndexesImpl() diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 18cf7cc67ab..9b0cc3cdea8 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1428,3 +1428,18 @@ def test_parallel_reading_with_memory_limit(started_cluster): # Check that server didn't crash result = instance.query("select 1") assert int(result) == 1 + + +def test_wrong_format_usage(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_wrong_format.native') select * from numbers(10)" + ) + + result = instance.query_and_get_error( + f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_wrong_format.native', 'Parquet') settings input_format_allow_seeks=0, max_memory_usage=1000" + ) + + assert "Not a Parquet file" in result From fd23c42ea743363f161c85dfb566b82f09b3dd85 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 13 Apr 2022 22:33:55 +0300 Subject: [PATCH 046/110] Update StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 243637d910b..db1de14f6a6 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1812,7 +1812,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che if (!parts_for_merge.empty() && replica.empty()) { - LOG_INFO(log, " {}. Will fetch merged part instead.", entry.new_part_name); + LOG_INFO(log, "No active replica has part {}. Will fetch merged part instead.", entry.new_part_name); /// We should enqueue it for check, because merged part may never appear if source part is lost enqueuePartForCheck(entry.new_part_name); return false; From b2cd23658a30b7d11ec17feacaffc83d7863ca1d Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 13 Apr 2022 22:45:34 +0300 Subject: [PATCH 047/110] Revert "Fix crash in ParallelReadBuffer" --- src/IO/ParallelReadBuffer.cpp | 6 +++++- tests/integration/test_storage_s3/test.py | 21 --------------------- 2 files changed, 5 insertions(+), 22 deletions(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 79a16347094..f036d6a08c8 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -33,7 +33,6 @@ bool ParallelReadBuffer::addReaderToPool(std::unique_lock & /*buffer auto worker = read_workers.emplace_back(std::make_shared(std::move(reader))); - ++active_working_reader; schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }); return true; @@ -204,6 +203,11 @@ bool ParallelReadBuffer::nextImpl() void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) { + { + std::lock_guard lock{mutex}; + ++active_working_reader; + } + SCOPE_EXIT({ std::lock_guard lock{mutex}; --active_working_reader; diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 18cf7cc67ab..e32ddd2782b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1407,24 +1407,3 @@ def test_insert_select_schema_inference(started_cluster): f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_insert_select.native')" ) assert int(result) == 1 - - -def test_parallel_reading_with_memory_limit(started_cluster): - bucket = started_cluster.minio_bucket - instance = started_cluster.instances["dummy"] - - instance.query( - f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') select * from numbers(100000)" - ) - - result = instance.query_and_get_error( - f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') settings max_memory_usage=10000" - ) - - assert "Memory limit (for query) exceeded" in result - - sleep(5) - - # Check that server didn't crash - result = instance.query("select 1") - assert int(result) == 1 From 6e89fc4542bc08e269e6fe3fcc082da83b48b5ed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Apr 2022 23:29:53 +0300 Subject: [PATCH 048/110] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index a8ca2079070..26d61dabaf9 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -3,13 +3,10 @@ toc_priority: 66 toc_title: ClickHouse Keeper --- -# [pre-production] ClickHouse Keeper {#clickHouse-keeper} +# ClickHouse Keeper {#clickHouse-keeper} ClickHouse server uses [ZooKeeper](https://zookeeper.apache.org/) coordination system for data [replication](../engines/table-engines/mergetree-family/replication.md) and [distributed DDL](../sql-reference/distributed-ddl.md) queries execution. ClickHouse Keeper is an alternative coordination system compatible with ZooKeeper. -!!! warning "Warning" - This feature is currently in the pre-production stage. We test it in our CI and on small internal installations. - ## Implementation details {#implementation-details} ZooKeeper is one of the first well-known open-source coordination systems. It's implemented in Java, has quite a simple and powerful data model. ZooKeeper's coordination algorithm called ZAB (ZooKeeper Atomic Broadcast) doesn't provide linearizability guarantees for reads, because each ZooKeeper node serves reads locally. Unlike ZooKeeper ClickHouse Keeper is written in C++ and uses [RAFT algorithm](https://raft.github.io/) [implementation](https://github.com/eBay/NuRaft). This algorithm allows to have linearizability for reads and writes, has several open-source implementations in different languages. From ca14a29dbeb6a01303986b6fb5a127db66cc3cf4 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 13 Apr 2022 20:39:12 +0000 Subject: [PATCH 049/110] Revert reverting "Fix crash in ParallelReadBuffer" --- src/IO/ParallelReadBuffer.cpp | 6 +----- tests/integration/test_storage_s3/test.py | 21 +++++++++++++++++++++ 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index f036d6a08c8..79a16347094 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -33,6 +33,7 @@ bool ParallelReadBuffer::addReaderToPool(std::unique_lock & /*buffer auto worker = read_workers.emplace_back(std::make_shared(std::move(reader))); + ++active_working_reader; schedule([this, worker = std::move(worker)]() mutable { readerThreadFunction(std::move(worker)); }); return true; @@ -203,11 +204,6 @@ bool ParallelReadBuffer::nextImpl() void ParallelReadBuffer::readerThreadFunction(ReadWorkerPtr read_worker) { - { - std::lock_guard lock{mutex}; - ++active_working_reader; - } - SCOPE_EXIT({ std::lock_guard lock{mutex}; --active_working_reader; diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index e32ddd2782b..87f97e7454a 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1407,3 +1407,24 @@ def test_insert_select_schema_inference(started_cluster): f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_insert_select.native')" ) assert int(result) == 1 + + +def test_parallel_reading_with_memory_limit(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') select * from numbers(1000000)" + ) + + result = instance.query_and_get_error( + f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_memory_limit.native') settings max_memory_usage=1000" + ) + + assert "Memory limit (for query) exceeded" in result + + time.sleep(5) + + # Check that server didn't crash + result = instance.query("select 1") + assert int(result) == 1 From 01a1babd3b4966678144fe045f64b779f44307fb Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 13 Apr 2022 23:56:34 +0300 Subject: [PATCH 050/110] Update ci_config.py --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b4f13817896..a4847f33d45 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -282,6 +282,7 @@ CI_CONFIG = { }, "Stateless tests (release, s3 storage, actions)": { "required_build": "package_release", + "force_tests": True, }, "Stress test (address, actions)": { "required_build": "package_asan", From 0595b5c22b98324089633d0d75ad59ebe78ee768 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 13 Apr 2022 21:50:57 +0000 Subject: [PATCH 051/110] fix reading of empty arrays in reverse order --- .../CompressedReadBufferFromFile.cpp | 2 +- .../02267_empty_arrays_read_reverse.reference | 1 + .../02267_empty_arrays_read_reverse.sql | 22 +++++++++++++++++++ 3 files changed, 24 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02267_empty_arrays_read_reverse.reference create mode 100644 tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 9efb3c92cde..a959da2c3ae 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -62,7 +62,7 @@ void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t { /// Nothing to do if we already at required position if (!size_compressed && static_cast(file_in.getPosition()) == offset_in_compressed_file && /// correct position in compressed file - (offset() == offset_in_decompressed_block /// correct position in buffer or + ((!buffer().empty() && offset() == offset_in_decompressed_block) /// correct position in buffer or || nextimpl_working_buffer_offset == offset_in_decompressed_block)) /// we will move our position to correct one return; diff --git a/tests/queries/0_stateless/02267_empty_arrays_read_reverse.reference b/tests/queries/0_stateless/02267_empty_arrays_read_reverse.reference new file mode 100644 index 00000000000..cc0cc5714d9 --- /dev/null +++ b/tests/queries/0_stateless/02267_empty_arrays_read_reverse.reference @@ -0,0 +1 @@ +['x'] 0 ['1','2','3','4','5','6'] diff --git a/tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql b/tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql new file mode 100644 index 00000000000..7c1cf47c540 --- /dev/null +++ b/tests/queries/0_stateless/02267_empty_arrays_read_reverse.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS t_02267; + +CREATE TABLE t_02267 +( + a Array(String), + b UInt32, + c Array(String) +) +ENGINE = MergeTree +ORDER BY b +SETTINGS index_granularity = 500; + +INSERT INTO t_02267 (b, a, c) SELECT 0, ['x'], ['1','2','3','4','5','6'] FROM numbers(1) ; +INSERT INTO t_02267 (b, a, c) SELECT 1, [], ['1','2','3','4','5','6'] FROM numbers(300000); + +OPTIMIZE TABLE t_02267 FINAL; + +SELECT * FROM t_02267 WHERE hasAll(a, ['x']) +ORDER BY b DESC +SETTINGS max_threads=1, max_block_size=1000; + +DROP TABLE IF EXISTS t_02267; From 4cce325941d7bc5df35a6ee7d5ec2b2449c22715 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 14 Apr 2022 00:25:14 +0200 Subject: [PATCH 052/110] Add Tyler Hannan to contributors --- tests/ci/run_check.py | 1 + tests/ci/workflow_approve_rerun_lambda/app.py | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index d76e28f96f9..183e47bbc87 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -80,6 +80,7 @@ TRUSTED_CONTRIBUTORS = { "thomoco", # ClickHouse "BoloniniD", # Seasoned contributor, HSE "tonickkozlov", # Cloudflare + "tylerhannan", # ClickHouse Employee ] } diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index b650d1651fe..7825bad141f 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -121,6 +121,7 @@ TRUSTED_CONTRIBUTORS = { "YiuRULE", "zlobober", # Developer of YT "BoloniniD", # Seasoned contributor, HSE + "tylerhannan", # ClickHouse Employee ] } From d2a7c4541edfa8e58eb8e0c84596488797b55e7b Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Thu, 14 Apr 2022 00:36:38 +0200 Subject: [PATCH 053/110] whitespace --- tests/ci/workflow_approve_rerun_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 7825bad141f..79e6e300bca 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -121,7 +121,7 @@ TRUSTED_CONTRIBUTORS = { "YiuRULE", "zlobober", # Developer of YT "BoloniniD", # Seasoned contributor, HSE - "tylerhannan", # ClickHouse Employee + "tylerhannan", # ClickHouse Employee ] } From 281ce4bfc746c106be4334a60584ae8a0d8118d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Apr 2022 03:04:34 +0300 Subject: [PATCH 054/110] Update http.md --- docs/en/interfaces/http.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index d72fb4d6f17..ca2a165bbd8 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -5,11 +5,10 @@ toc_title: HTTP Interface # HTTP Interface {#http-interface} -The HTTP interface lets you use ClickHouse on any platform from any programming language. We use it for working from Java and Perl, as well as shell scripts. In other departments, the HTTP interface is used from Perl, Python, and Go. The HTTP interface is more limited than the native interface, but it has better compatibility. +The HTTP interface lets you use ClickHouse on any platform from any programming language in a form of REST API. The HTTP interface is more limited than the native interface, but it has better language support. By default, `clickhouse-server` listens for HTTP on port 8123 (this can be changed in the config). - -Sometimes, `curl` command is not available on user operating systems. On Ubuntu or Debian, run `sudo apt install curl`. Please refer this [documentation](https://curl.se/download.html) to install it before running the examples. +HTTPS can be enabled as well with port 8443 by default. If you make a `GET /` request without parameters, it returns 200 response code and the string which defined in [http_server_default_response](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-http_server_default_response) default value “Ok.” (with a line feed at the end) @@ -18,11 +17,12 @@ $ curl 'http://localhost:8123/' Ok. ``` +Sometimes, `curl` command is not available on user operating systems. On Ubuntu or Debian, run `sudo apt install curl`. Please refer this [documentation](https://curl.se/download.html) to install it before running the examples. + Web UI can be accessed here: `http://localhost:8123/play`. ![Web UI](../images/play.png) - In health-check scripts use `GET /ping` request. This handler always returns “Ok.” (with a line feed at the end). Available from version 18.12.13. See also `/replicas_status` to check replica's delay. ``` bash @@ -32,7 +32,7 @@ $ curl 'http://localhost:8123/replicas_status' Ok. ``` -Send the request as a URL ‘query’ parameter, or as a POST. Or send the beginning of the query in the ‘query’ parameter, and the rest in the POST (we’ll explain later why this is necessary). The size of the URL is limited to 16 KB, so keep this in mind when sending large queries. +Send the request as a URL ‘query’ parameter, or as a POST. Or send the beginning of the query in the ‘query’ parameter, and the rest in the POST (we’ll explain later why this is necessary). The size of the URL is limited to 1 MiB by default, this can be changed with the `http_max_uri_size` setting. If successful, you receive the 200 response code and the result in the response body. If an error occurs, you receive the 500 response code and an error description text in the response body. From 427c73e7a52655246da99aa0ef981b1101e23b95 Mon Sep 17 00:00:00 2001 From: jewisliu Date: Wed, 13 Apr 2022 10:55:45 +0800 Subject: [PATCH 055/110] fix Enum overflow --- src/DataTypes/DataTypeEnum.cpp | 13 +++++++++++-- tests/queries/0_stateless/00757_enum_defaults.sql | 4 ++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index e292897e451..9d1424d862b 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -193,19 +193,28 @@ static void checkASTStructure(const ASTPtr & child) static void autoAssignNumberForEnum(const ASTPtr & arguments) { - Int16 child_count = 1; + UInt64 literal_child_count = 0; + UInt64 func_child_count = 0; ASTs assign_number_child; assign_number_child.reserve(arguments->children.size()); for (const ASTPtr & child : arguments->children) { if (child->as()) { - ASTPtr func = makeASTFunction("equals", child, std::make_shared(child_count++)); + ASTPtr func = makeASTFunction("equals", child, std::make_shared(++literal_child_count)); assign_number_child.emplace_back(func); } else + { + ++func_child_count; assign_number_child.emplace_back(child); + } } + + if (func_child_count > 0 && literal_child_count > 0) + throw Exception("ALL Elements of Enum data type must be of form: 'name' = number or 'name', where name is string literal and number is an integer", + ErrorCodes::UNEXPECTED_AST_STRUCTURE); + arguments->children = assign_number_child; } diff --git a/tests/queries/0_stateless/00757_enum_defaults.sql b/tests/queries/0_stateless/00757_enum_defaults.sql index 01f795b31f9..769579ffc0b 100644 --- a/tests/queries/0_stateless/00757_enum_defaults.sql +++ b/tests/queries/0_stateless/00757_enum_defaults.sql @@ -2,7 +2,11 @@ select os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'andro select toNullable(os_name) AS os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; DROP TABLE IF EXISTS auto_assgin_enum; +DROP TABLE IF EXISTS auto_assgin_enum1; + CREATE TABLE auto_assgin_enum (x enum('a', 'b')) ENGINE=MergeTree() order by x; +CREATE TABLE auto_assgin_enum1 (x enum('a' = 1, 'b')) ENGINE=MergeTree() order by x; -- { serverError 223 } INSERT INTO auto_assgin_enum VALUES('a'), ('b'); select * from auto_assgin_enum; + DROP TABLE auto_assgin_enum; From 323cf6c5bf25cbaa350d0e1a0a194642e6790485 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Apr 2022 13:01:25 +0200 Subject: [PATCH 056/110] Merge TRUSTED_CONTRIBUTORS in lambda and import in check --- tests/ci/run_check.py | 74 ++----------------- .../workflow_approve_rerun_lambda/__init__.py | 1 + tests/ci/workflow_approve_rerun_lambda/app.py | 5 ++ 3 files changed, 14 insertions(+), 66 deletions(-) create mode 100644 tests/ci/workflow_approve_rerun_lambda/__init__.py diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 183e47bbc87..69df4a87563 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -5,10 +5,12 @@ import re from typing import Tuple from github import Github -from env_helper import GITHUB_RUN_URL, GITHUB_REPOSITORY, GITHUB_SERVER_URL -from pr_info import PRInfo -from get_robot_token import get_best_robot_token + from commit_status_helper import get_commit, post_labels, remove_labels +from env_helper import GITHUB_RUN_URL, GITHUB_REPOSITORY, GITHUB_SERVER_URL +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from workflow_approve_rerun_lambda.app import TRUSTED_CONTRIBUTORS NAME = "Run Check (actions)" @@ -24,70 +26,12 @@ DO_NOT_TEST_LABEL = "do not test" FORCE_TESTS_LABEL = "force tests" SUBMODULE_CHANGED_LABEL = "submodule changed" -# Individual trusted contirbutors who are not in any trusted organization. -# Can be changed in runtime: we will append users that we learned to be in -# a trusted org, to save GitHub API calls. -TRUSTED_CONTRIBUTORS = { - e.lower() - for e in [ - "achimbab", - "adevyatova ", # DOCSUP - "Algunenano", # Raúl Marín, Tinybird - "amosbird", - "AnaUvarova", # DOCSUP - "anauvarova", # technical writer, Yandex - "annvsh", # technical writer, Yandex - "atereh", # DOCSUP - "azat", - "bharatnc", # Newbie, but already with many contributions. - "bobrik", # Seasoned contributor, CloudFlare - "BohuTANG", - "codyrobert", # Flickerbox engineer - "cwurm", # Employee - "damozhaeva", # DOCSUP - "den-crane", - "flickerbox-tom", # Flickerbox - "gyuton", # technical writer, Yandex - "hagen1778", # Roman Khavronenko, seasoned contributor - "hczhcz", - "hexiaoting", # Seasoned contributor - "ildus", # adjust, ex-pgpro - "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto - "ka1bi4", # DOCSUP - "kirillikoff", # DOCSUP - "kitaisreal", # Seasoned contributor - "kreuzerkrieg", - "lehasm", # DOCSUP - "michon470", # DOCSUP - "MyroTk", # Tester in Altinity - "myrrc", # Michael Kot, Altinity - "nikvas0", - "nvartolomei", - "olgarev", # DOCSUP - "otrazhenia", # Yandex docs contractor - "pdv-ru", # DOCSUP - "podshumok", # cmake expert from QRator Labs - "s-mx", # Maxim Sabyanin, former employee, present contributor - "sevirov", # technical writer, Yandex - "spongedu", # Seasoned contributor - "taiyang-li", - "ucasFL", # Amos Bird's friend - "vdimir", # Employee - "vzakaznikov", - "YiuRULE", - "zlobober", # Developer of YT - "ilejn", # Arenadata, responsible for Kerberized Kafka - "thomoco", # ClickHouse - "BoloniniD", # Seasoned contributor, HSE - "tonickkozlov", # Cloudflare - "tylerhannan", # ClickHouse Employee - ] -} MAP_CATEGORY_TO_LABEL = { "New Feature": "pr-feature", "Bug Fix": "pr-bugfix", - "Bug Fix (user-visible misbehaviour in official stable or prestable release)": "pr-bugfix", + "Bug Fix (user-visible misbehaviour in official " + "stable or prestable release)": "pr-bugfix", "Improvement": "pr-improvement", "Performance Improvement": "pr-performance", "Backward Incompatible Change": "pr-backward-incompatible", @@ -257,9 +201,7 @@ if __name__ == "__main__": elif SUBMODULE_CHANGED_LABEL in pr_info.labels: pr_labels_to_remove.append(SUBMODULE_CHANGED_LABEL) - print( - "change labels: add {}, remove {}".format(pr_labels_to_add, pr_labels_to_remove) - ) + print(f"change labels: add {pr_labels_to_add}, remove {pr_labels_to_remove}") if pr_labels_to_add: post_labels(gh, pr_info, pr_labels_to_add) diff --git a/tests/ci/workflow_approve_rerun_lambda/__init__.py b/tests/ci/workflow_approve_rerun_lambda/__init__.py new file mode 100644 index 00000000000..4265cc3e6c1 --- /dev/null +++ b/tests/ci/workflow_approve_rerun_lambda/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 79e6e300bca..98a686fe3f9 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -89,9 +89,11 @@ TRUSTED_CONTRIBUTORS = { "bharatnc", # Newbie, but already with many contributions. "bobrik", # Seasoned contributor, CloudFlare "BohuTANG", + "codyrobert", # Flickerbox engineer "cwurm", # Employee "damozhaeva", # DOCSUP "den-crane", + "flickerbox-tom", # Flickerbox "gyuton", # DOCSUP "hagen1778", # Roman Khavronenko, seasoned contributor "hczhcz", @@ -120,7 +122,10 @@ TRUSTED_CONTRIBUTORS = { "vzakaznikov", "YiuRULE", "zlobober", # Developer of YT + "ilejn", # Arenadata, responsible for Kerberized Kafka + "thomoco", # ClickHouse "BoloniniD", # Seasoned contributor, HSE + "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse Employee ] } From 5f31185125ac96b67dffa0911cbe7a09e5da53f8 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 14 Apr 2022 14:35:46 +0300 Subject: [PATCH 057/110] Update stress --- docker/test/stress/stress | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index e42587e9a4d..d78de84f60d 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -83,15 +83,15 @@ def make_query_command(query): def prepare_for_hung_check(drop_databases): # FIXME this function should not exist, but... - # ThreadFuzzer significantly slows down server and causes false-positive hung check failures - call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'") - # We attach gdb to clickhouse-server before running tests # to print stacktraces of all crashes even if clickhouse cannot print it for some reason. # However, it obstruct checking for hung queries. logging.info("Will terminate gdb (if any)") call_with_retry("kill -TERM $(pidof gdb)") + # ThreadFuzzer significantly slows down server and causes false-positive hung check failures + call_with_retry("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'") + call_with_retry(make_query_command('SELECT 1 FORMAT Null')) # Some tests execute SYSTEM STOP MERGES or similar queries. From 3e42a1711dfec2a77dfe92d945b97e52526745b2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 14 Apr 2022 13:17:04 +0200 Subject: [PATCH 058/110] Fix --- src/Common/FileSegment.cpp | 30 ++++++++++++++----- src/Common/FileSegment.h | 1 + src/Disks/IO/CachedReadBufferFromRemoteFS.cpp | 3 ++ 3 files changed, 26 insertions(+), 8 deletions(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 50a1ecb4a6c..3729b87e823 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -163,6 +163,11 @@ bool FileSegment::isDownloader() const return getCallerId() == downloader_id; } +bool FileSegment::isDownloaderImpl(std::lock_guard & /* segment+_lock */) const +{ + return getCallerId() == downloader_id; +} + FileSegment::RemoteFileReaderPtr FileSegment::getRemoteFileReader() { if (!isDownloader()) @@ -397,6 +402,9 @@ bool FileSegment::reserve(size_t size) void FileSegment::setDownloaded(std::lock_guard & /* segment_lock */) { + if (is_downloaded) + return; + download_state = State::DOWNLOADED; is_downloaded = true; downloader_id.clear(); @@ -426,8 +434,7 @@ void FileSegment::completeBatchAndResetDownloader() { std::lock_guard segment_lock(mutex); - bool is_downloader = downloader_id == getCallerId(); - if (!is_downloader) + if (!isDownloaderImpl(segment_lock)) { cv.notify_all(); throw Exception( @@ -448,7 +455,7 @@ void FileSegment::complete(State state) std::lock_guard cache_lock(cache->mutex); std::lock_guard segment_lock(mutex); - bool is_downloader = downloader_id == getCallerId(); + bool is_downloader = isDownloaderImpl(segment_lock); if (!is_downloader) { cv.notify_all(); @@ -465,6 +472,9 @@ void FileSegment::complete(State state) "Cannot complete file segment with state: {}", stateToString(state)); } + if (state == State::DOWNLOADED) + setDownloaded(segment_lock); + download_state = state; assertNotDetached(); @@ -475,7 +485,7 @@ void FileSegment::complete(State state) } catch (...) { - if (!downloader_id.empty() && downloader_id == getCallerIdImpl()) + if (!downloader_id.empty() && is_downloader) downloader_id.clear(); cv.notify_all(); @@ -492,8 +502,12 @@ void FileSegment::complete(std::lock_guard & cache_lock) if (download_state == State::SKIP_CACHE || detached) return; - if (download_state != State::DOWNLOADED && getDownloadedSize(segment_lock) == range().size()) + if (isDownloaderImpl(segment_lock) + && download_state != State::DOWNLOADED + && getDownloadedSize(segment_lock) == range().size()) + { setDownloaded(segment_lock); + } assertNotDetached(); @@ -502,7 +516,7 @@ void FileSegment::complete(std::lock_guard & cache_lock) /// Segment state can be changed from DOWNLOADING or EMPTY only if the caller is the /// downloader or the only owner of the segment. - bool can_update_segment_state = downloader_id == getCallerIdImpl() + bool can_update_segment_state = isDownloaderImpl(segment_lock) || cache->isLastFileSegmentHolder(key(), offset(), cache_lock, segment_lock); if (can_update_segment_state) @@ -515,7 +529,7 @@ void FileSegment::complete(std::lock_guard & cache_lock) } catch (...) { - if (!downloader_id.empty() && downloader_id == getCallerIdImpl()) + if (!downloader_id.empty() && isDownloaderImpl(segment_lock)) downloader_id.clear(); cv.notify_all(); @@ -561,7 +575,7 @@ void FileSegment::completeImpl(std::lock_guard & cache_lock, std::lo } } - if (!downloader_id.empty() && (downloader_id == getCallerIdImpl() || is_last_holder)) + if (!downloader_id.empty() && (isDownloaderImpl(segment_lock) || is_last_holder)) { LOG_TEST(log, "Clearing downloader id: {}, current state: {}", downloader_id, stateToString(download_state)); downloader_id.clear(); diff --git a/src/Common/FileSegment.h b/src/Common/FileSegment.h index 8439389fdeb..e3011eeb0fa 100644 --- a/src/Common/FileSegment.h +++ b/src/Common/FileSegment.h @@ -154,6 +154,7 @@ private: void setDownloaded(std::lock_guard & segment_lock); void setDownloadFailed(std::lock_guard & segment_lock); + bool isDownloaderImpl(std::lock_guard & segment_lock) const; void wrapWithCacheInfo(Exception & e, const String & message, std::lock_guard & segment_lock) const; diff --git a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp index 90a1d4ad43e..e6188a96a33 100644 --- a/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp +++ b/src/Disks/IO/CachedReadBufferFromRemoteFS.cpp @@ -613,7 +613,10 @@ bool CachedReadBufferFromRemoteFS::nextImplStep() { bool need_complete_file_segment = file_segment->isDownloader(); if (need_complete_file_segment) + { + LOG_TEST(log, "Resetting downloader {} from scope exit", file_segment->getDownloader()); file_segment->completeBatchAndResetDownloader(); + } } catch (...) { From 428753a4a77135ad76a2b5f76f0b9c38bb5e74b6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Apr 2022 14:39:36 +0200 Subject: [PATCH 059/110] Fix integration tests parser --- tests/integration/ci-runner.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 05e56d2a910..a047c6c114c 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -461,6 +461,11 @@ class ClickhouseIntegrationTestsRunner: if test not in main_counters[state]: main_counters[state].append(test) + for state in ("SKIPPED",): + for test in current_counters[state]: + main_counters[state].append(test) + + def _get_runner_image_cmd(self, repo_path): image_cmd = "" if self._can_run_with( From c74b6c547b6638648e1df14a6dd5478d993c5333 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Apr 2022 14:40:56 +0200 Subject: [PATCH 060/110] Black --- tests/integration/ci-runner.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index a047c6c114c..687c88b04a3 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -465,7 +465,6 @@ class ClickhouseIntegrationTestsRunner: for test in current_counters[state]: main_counters[state].append(test) - def _get_runner_image_cmd(self, repo_path): image_cmd = "" if self._can_run_with( From 9947008d31f7b5031ca8170438853cb0f2e7be23 Mon Sep 17 00:00:00 2001 From: zhanghuajie <310336422@qq.com> Date: Thu, 14 Apr 2022 22:49:49 +0800 Subject: [PATCH 061/110] fix crash when you use clickhouse-git-import with invalid parameter for example: clickhouse-git-import -xxx --- programs/compressor/Compressor.cpp | 64 +++++++++++++++--------------- programs/format/Format.cpp | 64 +++++++++++++++--------------- programs/git-import/git-import.cpp | 2 +- 3 files changed, 65 insertions(+), 65 deletions(-) diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index d47372631fe..d0fc3528473 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -66,40 +66,40 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) using namespace DB; namespace po = boost::program_options; - po::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); - desc.add_options() - ("help,h", "produce help message") - ("input", po::value()->value_name("INPUT"), "input file") - ("output", po::value()->value_name("OUTPUT"), "output file") - ("decompress,d", "decompress") - ("offset-in-compressed-file", po::value()->default_value(0ULL), "offset to the compressed block (i.e. physical file offset)") - ("offset-in-decompressed-block", po::value()->default_value(0ULL), "offset to the decompressed block (i.e. virtual offset)") - ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") - ("hc", "use LZ4HC instead of LZ4") - ("zstd", "use ZSTD instead of LZ4") - ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") - ("level", po::value(), "compression level for codecs specified via flags") - ("none", "use no compression instead of LZ4") - ("stat", "print block statistics of compressed data") - ; - - po::positional_options_description positional_desc; - positional_desc.add("input", 1); - positional_desc.add("output", 1); - - po::variables_map options; - po::store(po::command_line_parser(argc, argv).options(desc).positional(positional_desc).run(), options); - - if (options.count("help")) - { - std::cout << "Usage: " << argv[0] << " [options] < INPUT > OUTPUT" << std::endl; - std::cout << "Usage: " << argv[0] << " [options] INPUT OUTPUT" << std::endl; - std::cout << desc << std::endl; - return 0; - } - try { + po::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); + desc.add_options() + ("help,h", "produce help message") + ("input", po::value()->value_name("INPUT"), "input file") + ("output", po::value()->value_name("OUTPUT"), "output file") + ("decompress,d", "decompress") + ("offset-in-compressed-file", po::value()->default_value(0ULL), "offset to the compressed block (i.e. physical file offset)") + ("offset-in-decompressed-block", po::value()->default_value(0ULL), "offset to the decompressed block (i.e. virtual offset)") + ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") + ("hc", "use LZ4HC instead of LZ4") + ("zstd", "use ZSTD instead of LZ4") + ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") + ("level", po::value(), "compression level for codecs specified via flags") + ("none", "use no compression instead of LZ4") + ("stat", "print block statistics of compressed data") + ; + + po::positional_options_description positional_desc; + positional_desc.add("input", 1); + positional_desc.add("output", 1); + + po::variables_map options; + po::store(po::command_line_parser(argc, argv).options(desc).positional(positional_desc).run(), options); + + if (options.count("help")) + { + std::cout << "Usage: " << argv[0] << " [options] < INPUT > OUTPUT" << std::endl; + std::cout << "Usage: " << argv[0] << " [options] INPUT OUTPUT" << std::endl; + std::cout << desc << std::endl; + return 0; + } + bool decompress = options.count("decompress"); bool use_lz4hc = options.count("hc"); bool use_zstd = options.count("zstd"); diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 50d85cdd43d..d5206da00f5 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -44,40 +44,40 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { using namespace DB; - boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); - desc.add_options() - ("query", po::value(), "query to format") - ("help,h", "produce help message") - ("hilite", "add syntax highlight with ANSI terminal escape sequences") - ("oneline", "format in single line") - ("quiet,q", "just check syntax, no output on success") - ("multiquery,n", "allow multiple queries in the same file") - ("obfuscate", "obfuscate instead of formatting") - ("backslash", "add a backslash at the end of each line of the formatted query") - ("allow_settings_after_format_in_insert", "Allow SETTINGS after FORMAT, but note, that this is not always safe") - ("seed", po::value(), "seed (arbitrary string) that determines the result of obfuscation") - ; - - Settings cmd_settings; - for (const auto & field : cmd_settings.all()) - { - if (field.getName() == "max_parser_depth" || field.getName() == "max_query_size") - cmd_settings.addProgramOption(desc, field); - } - - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - po::notify(options); - - if (options.count("help")) - { - std::cout << "Usage: " << argv[0] << " [options] < query" << std::endl; - std::cout << desc << std::endl; - return 1; - } - try { + boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); + desc.add_options() + ("query", po::value(), "query to format") + ("help,h", "produce help message") + ("hilite", "add syntax highlight with ANSI terminal escape sequences") + ("oneline", "format in single line") + ("quiet,q", "just check syntax, no output on success") + ("multiquery,n", "allow multiple queries in the same file") + ("obfuscate", "obfuscate instead of formatting") + ("backslash", "add a backslash at the end of each line of the formatted query") + ("allow_settings_after_format_in_insert", "Allow SETTINGS after FORMAT, but note, that this is not always safe") + ("seed", po::value(), "seed (arbitrary string) that determines the result of obfuscation") + ; + + Settings cmd_settings; + for (const auto & field : cmd_settings.all()) + { + if (field.getName() == "max_parser_depth" || field.getName() == "max_query_size") + cmd_settings.addProgramOption(desc, field); + } + + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + po::notify(options); + + if (options.count("help")) + { + std::cout << "Usage: " << argv[0] << " [options] < query" << std::endl; + std::cout << desc << std::endl; + return 1; + } + bool hilite = options.count("hilite"); bool oneline = options.count("oneline"); bool quiet = options.count("quiet"); diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index 749dcbfee5f..18a9bb2627c 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -1231,5 +1231,5 @@ try catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; - throw; + return DB::getCurrentExceptionCode(); } From 52a7544aa48d8978b072b991d612e557a3ff20e4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 13 Apr 2022 18:48:56 +0200 Subject: [PATCH 062/110] Fix cancel-lambda for closed PRs - Create a fallback function for some edge cases - Process synchronized PRs as help for cancel-workflow action --- .../cancel_and_rerun_workflow_lambda/app.py | 114 +++++++++++++++--- 1 file changed, 94 insertions(+), 20 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 4cce1e5024a..663d9100889 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -92,28 +92,33 @@ WorkflowDescription = namedtuple( ) -def get_workflows_description_for_pull_request(pull_request_event): +def get_workflows_description_for_pull_request( + pull_request_event, +) -> List[WorkflowDescription]: head_repo = pull_request_event["head"]["repo"]["full_name"] head_branch = pull_request_event["head"]["ref"] - head_sha = pull_request_event["head"]["sha"] print("PR", pull_request_event["number"], "has head ref", head_branch) - workflows_data = [] - workflows = _exec_get_with_retry( - API_URL + f"/actions/runs?branch={head_branch}&event=pull_request&page=1" - ) - workflows_data += workflows["workflow_runs"] - i = 2 - while len(workflows["workflow_runs"]) > 0: - workflows = _exec_get_with_retry( - API_URL + f"/actions/runs?branch={head_branch}&event=pull_request&page={i}" - ) - workflows_data += workflows["workflow_runs"] - i += 1 - if i > 30: - print("Too many workflows found") - break - DEBUG_INFO["workflows"] = [] # type: List[Dict[str, str]] + workflows_data = [] + request_url = f"{API_URL}/actions/runs?per_page=100" + # Get all workflows for the current branch + for i in range(1, 11): + workflows = _exec_get_with_retry( + f"{request_url}&event=pull_request&branch={head_branch}&page={i}" + ) + if not workflows["workflow_runs"]: + break + workflows_data += workflows["workflow_runs"] + if i == 10: + print("Too many workflows found") + + if not workflows_data: + print("No workflows found by filter") + return [] + + print(f"Total workflows for the branch {head_branch} found: {len(workflows_data)}") + + DEBUG_INFO["workflows"] = [] workflow_descriptions = [] for workflow in workflows_data: # Some time workflow["head_repository"]["full_name"] is None @@ -123,13 +128,13 @@ def get_workflows_description_for_pull_request(pull_request_event): { "full_name": workflow["head_repository"]["full_name"], "name": workflow["name"], + "branch": workflow["head_branch"], } ) # unfortunately we cannot filter workflows from forks in request to API # so doing it manually if ( - workflow["head_sha"] == head_sha - and workflow["head_repository"]["full_name"] == head_repo + workflow["head_repository"]["full_name"] == head_repo and workflow["name"] in NEED_RERUN_OR_CANCELL_WORKFLOWS ): workflow_descriptions.append( @@ -144,6 +149,60 @@ def get_workflows_description_for_pull_request(pull_request_event): return workflow_descriptions +def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]: + pull_request_event = event_data["pull_request"] + head_repo = pull_request_event["head"]["repo"]["full_name"] + head_branch = pull_request_event["head"]["ref"] + head_sha = pull_request_event["head"]["sha"] + print("Get last 500 workflows from API to search related there") + # Fallback for a case of an already deleted branch and no workflows received + request_url = f"{API_URL}/actions/runs?per_page=100" + workflows_data = [] + i = 1 + for i in range(1, 6): + workflows = _exec_get_with_retry(f"{request_url}&page={i}") + if not workflows["workflow_runs"]: + break + # Prefilter workflows + workflows_data += [ + wf + for wf in workflows["workflow_runs"] + if wf["head_repository"] is not None + and wf["head_repository"]["full_name"] == head_repo + and wf["head_branch"] == head_branch + and wf["name"] in NEED_RERUN_OR_CANCELL_WORKFLOWS + ] + + print(f"Total workflows in last 500 actions matches: {len(workflows_data)}") + + DEBUG_INFO["workflows"] = [ + { + "full_name": wf["head_repository"]["full_name"], + "name": wf["name"], + "branch": wf["head_branch"], + } + for wf in workflows_data + ] + if event_data["action"] == "synchronize": + print(f"Leave only workflows with SHA but {head_sha} for updated PR") + # Cancel all events with SHA different than current + workflows_data = list( + filter(lambda x: x["head_sha"] != head_sha, workflows_data) + ) + + workflow_descriptions = [ + WorkflowDescription( + run_id=wf["id"], + status=wf["status"], + rerun_url=wf["rerun_url"], + cancel_url=wf["cancel_url"], + ) + for wf in workflows_data + ] + + return workflow_descriptions + + def get_workflow_description(workflow_id): workflow = _exec_get_with_retry(API_URL + f"/actions/runs/{workflow_id}") return WorkflowDescription( @@ -189,6 +248,21 @@ def main(event): if action == "closed" or "do not test" in labels: print("PR merged/closed or manually labeled 'do not test' will kill workflows") workflow_descriptions = get_workflows_description_for_pull_request(pull_request) + workflow_descriptions = ( + workflow_descriptions or get_workflow_description_fallback(event_data) + ) + urls_to_cancel = [] + for workflow_description in workflow_descriptions: + if workflow_description.status != "completed": + urls_to_cancel.append(workflow_description.cancel_url) + print(f"Found {len(urls_to_cancel)} workflows to cancel") + exec_workflow_url(urls_to_cancel, token) + elif action == "synchronize": + print("PR is synchronized, going to stop old actions") + workflow_descriptions = get_workflows_description_for_pull_request(pull_request) + workflow_descriptions = ( + workflow_descriptions or get_workflow_description_fallback(event_data) + ) urls_to_cancel = [] for workflow_description in workflow_descriptions: if workflow_description.status != "completed": From 3e6dc24a7bbc6ac33e447018838dcc676ea8e769 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Apr 2022 19:35:29 +0200 Subject: [PATCH 063/110] Do not fail the whole lambda if one page failed --- tests/ci/cancel_and_rerun_workflow_lambda/app.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 663d9100889..e5decd5561b 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -160,7 +160,11 @@ def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]: workflows_data = [] i = 1 for i in range(1, 6): - workflows = _exec_get_with_retry(f"{request_url}&page={i}") + try: + workflows = _exec_get_with_retry(f"{request_url}&page={i}") + except Exception as e: + print(f"Exception occured, still continue: {e}") + continue if not workflows["workflow_runs"]: break # Prefilter workflows From 646fd45645951820544e08511728359e662c0c51 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Apr 2022 20:41:06 +0200 Subject: [PATCH 064/110] Request unfiltered workflows in threads --- .../cancel_and_rerun_workflow_lambda/app.py | 42 +++++++++++++++---- 1 file changed, 35 insertions(+), 7 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index e5decd5561b..cb68365d4d6 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -2,6 +2,8 @@ from collections import namedtuple from typing import Any, Dict, List +from threading import Thread +from queue import Queue import json import time @@ -25,6 +27,24 @@ MAX_RETRY = 5 DEBUG_INFO = {} # type: Dict[str, Any] +class Worker(Thread): + def __init__(self, request_queue: Queue, ignore_exception: bool = False): + Thread.__init__(self) + self.queue = request_queue + self.ignore_exception = ignore_exception + self.response = {} # type: Dict + + def run(self): + m = self.queue.get() + try: + self.response = _exec_get_with_retry(m) + except Exception as e: + if not self.ignore_exception: + raise + print(f"Exception occured, still continue: {e}") + self.queue.task_done() + + def get_installation_id(jwt_token): headers = { "Authorization": f"Bearer {jwt_token}", @@ -157,20 +177,25 @@ def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]: print("Get last 500 workflows from API to search related there") # Fallback for a case of an already deleted branch and no workflows received request_url = f"{API_URL}/actions/runs?per_page=100" + q = Queue() # type: Queue + workers = [] workflows_data = [] i = 1 for i in range(1, 6): - try: - workflows = _exec_get_with_retry(f"{request_url}&page={i}") - except Exception as e: - print(f"Exception occured, still continue: {e}") + q.put(f"{request_url}&page={i}") + worker = Worker(q, True) + worker.start() + workers.append(worker) + + for worker in workers: + worker.join() + if not worker.response: + # We ignore get errors, so response can be empty continue - if not workflows["workflow_runs"]: - break # Prefilter workflows workflows_data += [ wf - for wf in workflows["workflow_runs"] + for wf in worker.response["workflow_runs"] if wf["head_repository"] is not None and wf["head_repository"]["full_name"] == head_repo and wf["head_branch"] == head_branch @@ -276,6 +301,9 @@ def main(event): elif action == "labeled" and "can be tested" in labels: print("PR marked with can be tested label, rerun workflow") workflow_descriptions = get_workflows_description_for_pull_request(pull_request) + workflow_descriptions = ( + workflow_descriptions or get_workflow_description_fallback(event_data) + ) if not workflow_descriptions: print("Not found any workflows") return From fad0aeb0d34ed065b7b68ec1510000519231623d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Apr 2022 20:43:49 +0200 Subject: [PATCH 065/110] Fix some linter issues --- tests/ci/team_keys_lambda/app.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py index ad153664b86..50ef753001c 100644 --- a/tests/ci/team_keys_lambda/app.py +++ b/tests/ci/team_keys_lambda/app.py @@ -1,12 +1,13 @@ #!/usr/bin/env python3 -import requests import argparse import json from threading import Thread from queue import Queue +import requests # type: ignore + def get_org_team_members(token: str, org: str, team_slug: str) -> tuple: headers = { @@ -37,7 +38,7 @@ def get_members_keys(members: tuple) -> str: self.results.append(f"# {m}\n{response.text}") self.queue.task_done() - q = Queue() + q = Queue() # type: Queue workers = [] for m in members: q.put(m) @@ -61,7 +62,7 @@ def get_members_keys(members: tuple) -> str: def get_token_from_aws() -> str: - import boto3 + import boto3 # type: ignore secret_name = "clickhouse_robot_token" session = boto3.session.Session() @@ -81,6 +82,8 @@ def main(token: str, org: str, team_slug: str) -> str: def handler(event, context): + _ = context + _ = event token = get_token_from_aws() result = { "statusCode": 200, From e68932298036adc855e618514c1d3db7a1b777f0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Apr 2022 20:54:57 +0200 Subject: [PATCH 066/110] Improve logic to decide cancelling workflow --- .../cancel_and_rerun_workflow_lambda/app.py | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index cb68365d4d6..27aff07fc0b 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -108,7 +108,7 @@ def _exec_get_with_retry(url): WorkflowDescription = namedtuple( - "WorkflowDescription", ["run_id", "status", "rerun_url", "cancel_url"] + "WorkflowDescription", ["run_id", "status", "rerun_url", "cancel_url", "conclusion"] ) @@ -163,6 +163,7 @@ def get_workflows_description_for_pull_request( status=workflow["status"], rerun_url=workflow["rerun_url"], cancel_url=workflow["cancel_url"], + conclusion=workflow["conclusion"], ) ) @@ -225,6 +226,7 @@ def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]: status=wf["status"], rerun_url=wf["rerun_url"], cancel_url=wf["cancel_url"], + conclusion=wf["conclusion"], ) for wf in workflows_data ] @@ -232,13 +234,14 @@ def get_workflow_description_fallback(event_data) -> List[WorkflowDescription]: return workflow_descriptions -def get_workflow_description(workflow_id): +def get_workflow_description(workflow_id) -> WorkflowDescription: workflow = _exec_get_with_retry(API_URL + f"/actions/runs/{workflow_id}") return WorkflowDescription( run_id=workflow["id"], status=workflow["status"], rerun_url=workflow["rerun_url"], cancel_url=workflow["cancel_url"], + conclusion=workflow["conclusion"], ) @@ -282,7 +285,10 @@ def main(event): ) urls_to_cancel = [] for workflow_description in workflow_descriptions: - if workflow_description.status != "completed": + if ( + workflow_description.status != "completed" + and workflow_description.conclusion != "cancelled" + ): urls_to_cancel.append(workflow_description.cancel_url) print(f"Found {len(urls_to_cancel)} workflows to cancel") exec_workflow_url(urls_to_cancel, token) @@ -294,7 +300,10 @@ def main(event): ) urls_to_cancel = [] for workflow_description in workflow_descriptions: - if workflow_description.status != "completed": + if ( + workflow_description.status != "completed" + and workflow_description.conclusion != "cancelled" + ): urls_to_cancel.append(workflow_description.cancel_url) print(f"Found {len(urls_to_cancel)} workflows to cancel") exec_workflow_url(urls_to_cancel, token) @@ -311,7 +320,10 @@ def main(event): sorted_workflows = list(sorted(workflow_descriptions, key=lambda x: x.run_id)) most_recent_workflow = sorted_workflows[-1] print("Latest workflow", most_recent_workflow) - if most_recent_workflow.status != "completed": + if ( + most_recent_workflow.status != "completed" + and most_recent_workflow.conclusion != "cancelled" + ): print("Latest workflow is not completed, cancelling") exec_workflow_url([most_recent_workflow.cancel_url], token) print("Cancelled") From 401efa5fd47f2bec48bed57f6ccf0acab13d3a27 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 14 Apr 2022 22:59:03 +0200 Subject: [PATCH 067/110] Improve docker building and logging for approve lambda --- tests/ci/workflow_approve_rerun_lambda/Dockerfile | 6 +++--- tests/ci/workflow_approve_rerun_lambda/app.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/Dockerfile b/tests/ci/workflow_approve_rerun_lambda/Dockerfile index f53be71a893..0d50224c51d 100644 --- a/tests/ci/workflow_approve_rerun_lambda/Dockerfile +++ b/tests/ci/workflow_approve_rerun_lambda/Dockerfile @@ -1,13 +1,13 @@ FROM public.ecr.aws/lambda/python:3.9 -# Copy function code -COPY app.py ${LAMBDA_TASK_ROOT} - # Install the function's dependencies using file requirements.txt # from your project folder. COPY requirements.txt . RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}" +# Copy function code +COPY app.py ${LAMBDA_TASK_ROOT} + # Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile) CMD [ "app.handler" ] diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 98a686fe3f9..27e28dc30e0 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -394,7 +394,7 @@ def rerun_workflow(workflow_description, token): def main(event): token = get_token_from_aws() event_data = json.loads(event["body"]) - print("The body received:", event_data) + print("The body received:", event["body"]) workflow_description = get_workflow_description_from_event(event_data) print("Got workflow description", workflow_description) From dac469dfcab4b2cf04727e9fa6ef03368f7537a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Apr 2022 02:39:30 +0300 Subject: [PATCH 068/110] Update storing-data.md --- docs/en/operations/storing-data.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index beffd45bcbd..6cda47ab9fb 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -11,10 +11,6 @@ To work with data stored on `Amazon S3` disks use [S3](../engines/table-engines/ To load data from a web server with static files use a disk with type [web](#storing-data-on-webserver). -## Zero-copy Replication {#zero-copy} - -ClickHouse supports zero-copy replication for `S3` and `HDFS` disks, which means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. - ## Configuring HDFS {#configuring-hdfs} [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) and [Log](../engines/table-engines/log-family/log.md) family table engines can store data to HDFS using a disk with type `HDFS`. @@ -316,3 +312,8 @@ When loading files by `endpoint`, they must be loaded into `/store/` p If URL is not reachable on disk load when the server is starting up tables, then all errors are caught. If in this case there were errors, tables can be reloaded (become visible) via `DETACH TABLE table_name` -> `ATTACH TABLE table_name`. If metadata was successfully loaded at server startup, then tables are available straight away. Use [http_max_single_read_retries](../operations/settings/settings.md#http-max-single-read-retries) setting to limit the maximum number of retries during a single HTTP read. + + +## Zero-copy Replication (not ready for production) {#zero-copy} + +ClickHouse supports zero-copy replication for `S3` and `HDFS` disks, which means that if the data is stored remotely on several machines and needs to be synchronized, then only the metadata is replicated (paths to the data parts), but not the data itself. From efbb29539f62d14c6e620f87b43087964ccb23d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Apr 2022 03:13:40 +0200 Subject: [PATCH 069/110] Fix Play UI for some resultsets --- programs/server/play.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/play.html b/programs/server/play.html index ef45ee1476b..146316ef61f 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -589,7 +589,7 @@ stats.innerText = `Elapsed: ${seconds} sec, read ${formatted_rows} rows, ${formatted_bytes}.`; /// We can also render graphs if user performed EXPLAIN PIPELINE graph=1 or EXPLAIN AST graph = 1 - if (response.data.length > 3 && response.data[0][0].startsWith("digraph") && document.getElementById('query').value.match(/^\s*EXPLAIN/i)) { + if (response.data.length > 3 && document.getElementById('query').value.match(/^\s*EXPLAIN/i) && typeof(response.data[0][0]) === "string" && response.data[0][0].startsWith("digraph")) { renderGraph(response); } else { renderTable(response); From 40e5ba21a4265560fa2e46c5c90594832bd316f4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Apr 2022 11:55:05 +0200 Subject: [PATCH 070/110] Update src/Common/FileSegment.cpp --- src/Common/FileSegment.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 3729b87e823..9fe95c0d8cd 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -163,7 +163,7 @@ bool FileSegment::isDownloader() const return getCallerId() == downloader_id; } -bool FileSegment::isDownloaderImpl(std::lock_guard & /* segment+_lock */) const +bool FileSegment::isDownloaderImpl(std::lock_guard & /* segment_lock */) const { return getCallerId() == downloader_id; } From b79d84acf2352f52afeea3dcf8692b9c0e66444b Mon Sep 17 00:00:00 2001 From: ifinik Date: Fri, 15 Apr 2022 23:39:21 +0300 Subject: [PATCH 071/110] =?UTF-8?q?=D0=BE=D0=BF=D0=B5=D1=87=D0=B0=D1=82?= =?UTF-8?q?=D0=BA=D0=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit https://www.google.com/search?q=%D0%BF%D0%BE%D0%BC%D0%B5%D1%89%D0%B0%D0%B5%D1%81%D1%8F+site%3Agithub.com&biw=1920&bih=969&ei=0NZZYv-UEOrhrgS76YvoDw&ved=0ahUKEwj_vK2w9Zb3AhXqsIsKHbv0Av0Q4dUDCA4&uact=5&oq=%D0%BF%D0%BE%D0%BC%D0%B5%D1%89%D0%B0%D0%B5%D1%81%D1%8F+site%3Agithub.com&gs_lcp=Cgdnd3Mtd2l6EANKBAhBGAFKBAhGGABQthhYwxtg7S1oAnAAeACAAUOIAXKSAQEymAEAoAEBwAEB&sclient=gws-wiz --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index c75fa8e92ce..f2a13569c23 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -182,7 +182,7 @@ Marks numbers: 0 1 2 3 4 5 6 7 8 Разреженный индекс допускает чтение лишних строк. При чтении одного диапазона первичного ключа, может быть прочитано до `index_granularity * 2` лишних строк в каждом блоке данных. -Разреженный индекс почти всегда помещаеся в оперативную память и позволяет работать с очень большим количеством строк в таблицах. +Разреженный индекс почти всегда помещается в оперативную память и позволяет работать с очень большим количеством строк в таблицах. ClickHouse не требует уникального первичного ключа. Можно вставить много строк с одинаковым первичным ключом. From 58fc4e033ae143af6a5802d89787e25dad2c2e8e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Apr 2022 22:54:08 +0200 Subject: [PATCH 072/110] Remove trash --- programs/server/Server.cpp | 1 + src/Common/getNumberOfPhysicalCPUCores.cpp | 94 ++++++++++------------ 2 files changed, 42 insertions(+), 53 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fc9187cb622..0120564e0b1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1716,6 +1716,7 @@ int Server::main(const std::vector & /*args*/) return Application::EXIT_OK; } + void Server::createServers( Poco::Util::AbstractConfiguration & config, const std::vector & listen_hosts, diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index 1d26b522e64..872a2464eb6 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -12,75 +12,63 @@ #include #if defined(OS_LINUX) -unsigned getCGroupLimitedCPUCores(unsigned default_cpu_count) +static int readFrom(const char * filename, int default_value) { - // Try to look at cgroups limit if it is available. - auto read_from = [](const char * filename, int default_value) -> int { - std::ifstream infile(filename); - if (!infile.is_open()) - { - return default_value; - } - int idata; - if (infile >> idata) - return idata; - else - return default_value; - }; + std::ifstream infile(filename); + if (!infile.is_open()) + return default_value; + int idata; + if (infile >> idata) + return idata; + else + return default_value; +} +/// Try to look at cgroups limit if it is available. +static unsigned getCGroupLimitedCPUCores(unsigned default_cpu_count) +{ unsigned quota_count = default_cpu_count; - // Return the number of milliseconds per period process is guaranteed to run. - // -1 for no quota - int cgroup_quota = read_from("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", -1); - int cgroup_period = read_from("/sys/fs/cgroup/cpu/cpu.cfs_period_us", -1); + /// Return the number of milliseconds per period process is guaranteed to run. + /// -1 for no quota + int cgroup_quota = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_quota_us", -1); + int cgroup_period = readFrom("/sys/fs/cgroup/cpu/cpu.cfs_period_us", -1); if (cgroup_quota > -1 && cgroup_period > 0) - { quota_count = ceil(static_cast(cgroup_quota) / static_cast(cgroup_period)); - } return std::min(default_cpu_count, quota_count); } -#endif // OS_LINUX +#endif unsigned getNumberOfPhysicalCPUCores() { - static const unsigned number = [] { - unsigned cpu_count = 0; // start with an invalid num + unsigned cpu_count = 0; // start with an invalid num + #if USE_CPUID - do - { - cpu_raw_data_t raw_data; - cpu_id_t data; + cpu_raw_data_t raw_data; + cpu_id_t data; - /// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method. - /// Also, libcpuid does not support some CPUs like AMD Hygon C86 7151. - if (0 != cpuid_get_raw_data(&raw_data) || 0 != cpu_identify(&raw_data, &data) || data.num_logical_cpus == 0) - { - // Just fallback - break; - } + /// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method. + /// Also, libcpuid does not support some CPUs like AMD Hygon C86 7151. + /// Also, libcpuid gives strange result on Google Compute Engine VMs. + /// Example: + /// num_cores = 12, /// number of physical cores on current CPU socket + /// total_logical_cpus = 1, /// total number of logical cores on all sockets + /// num_logical_cpus = 24. /// number of logical cores on current CPU socket + /// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1. - cpu_count = data.num_cores * data.total_logical_cpus / data.num_logical_cpus; - - /// Also, libcpuid gives strange result on Google Compute Engine VMs. - /// Example: - /// num_cores = 12, /// number of physical cores on current CPU socket - /// total_logical_cpus = 1, /// total number of logical cores on all sockets - /// num_logical_cpus = 24. /// number of logical cores on current CPU socket - /// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1. - } while (false); + if (0 == cpuid_get_raw_data(&raw_data) && 0 == cpu_identify(&raw_data, &data) && data.num_logical_cpus != 0) + cpu_count = data.num_cores * data.total_logical_cpus / data.num_logical_cpus; #endif - /// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system. - /// (Actually, only Aarch64 is supported). - if (cpu_count == 0) - cpu_count = std::thread::hardware_concurrency(); + /// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system. + /// (Actually, only Aarch64 is supported). + if (cpu_count == 0) + cpu_count = std::thread::hardware_concurrency(); #if defined(OS_LINUX) - /// TODO: add a setting for disabling that, similar to UseContainerSupport in java - cpu_count = getCGroupLimitedCPUCores(cpu_count); -#endif // OS_LINUX - return cpu_count; - }(); - return number; + /// TODO: add a setting for disabling that, similar to UseContainerSupport in java + cpu_count = getCGroupLimitedCPUCores(cpu_count); +#endif + + return cpu_count; } From 364f10aff5f0b11a45aeb24eb933f13a87fdef5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Apr 2022 22:56:51 +0200 Subject: [PATCH 073/110] Remove trash --- src/Common/getNumberOfPhysicalCPUCores.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index 872a2464eb6..f415ee58e85 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -39,7 +39,7 @@ static unsigned getCGroupLimitedCPUCores(unsigned default_cpu_count) } #endif -unsigned getNumberOfPhysicalCPUCores() +static unsigned getNumberOfPhysicalCPUCoresImpl() { unsigned cpu_count = 0; // start with an invalid num @@ -72,3 +72,10 @@ unsigned getNumberOfPhysicalCPUCores() return cpu_count; } + +unsigned getNumberOfPhysicalCPUCores() +{ + /// Calculate once. + static auto res = getNumberOfPhysicalCPUCoresImpl(); + return res; +} From 416fa95b8744825ec1c00c072b20bd404145198a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Apr 2022 23:57:09 +0200 Subject: [PATCH 074/110] Remove "Arcadia" build system --- src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h | 2 -- src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp | 2 -- src/IO/ReadBufferFromAzureBlobStorage.cpp | 2 -- src/IO/ReadBufferFromAzureBlobStorage.h | 2 -- src/IO/WriteBufferFromAzureBlobStorage.cpp | 2 -- src/IO/WriteBufferFromAzureBlobStorage.h | 2 -- src/Server/CertificateReloader.h | 4 +--- 7 files changed, 1 insertion(+), 15 deletions(-) diff --git a/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h index 1cef6105d41..048daa7c9dc 100644 --- a/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h +++ b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h @@ -1,8 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_AZURE_BLOB_STORAGE diff --git a/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 243452353d3..128c7534b3c 100644 --- a/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -1,6 +1,4 @@ -#if !defined(ARCADIA_BUILD) #include -#endif #include diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp index 0ce6db97437..593bd01023a 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -1,6 +1,4 @@ -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_AZURE_BLOB_STORAGE diff --git a/src/IO/ReadBufferFromAzureBlobStorage.h b/src/IO/ReadBufferFromAzureBlobStorage.h index 78d973747ba..d743e725bda 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/IO/ReadBufferFromAzureBlobStorage.h @@ -1,8 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_AZURE_BLOB_STORAGE diff --git a/src/IO/WriteBufferFromAzureBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp index 88882fcef65..0f8cfdf347d 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -1,6 +1,4 @@ -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_AZURE_BLOB_STORAGE diff --git a/src/IO/WriteBufferFromAzureBlobStorage.h b/src/IO/WriteBufferFromAzureBlobStorage.h index cbbfb577a91..75336c497eb 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/IO/WriteBufferFromAzureBlobStorage.h @@ -1,8 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_AZURE_BLOB_STORAGE diff --git a/src/Server/CertificateReloader.h b/src/Server/CertificateReloader.h index 88c732c2db6..f984b4e4691 100644 --- a/src/Server/CertificateReloader.h +++ b/src/Server/CertificateReloader.h @@ -1,8 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) -# include -#endif +#include #if USE_SSL From a0fe29e2433147756614133b1805703288aa077c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Apr 2022 23:59:49 +0200 Subject: [PATCH 075/110] Remove Arcadia --- src/Interpreters/ActionsVisitor.cpp | 18 ++++++++++++++++++ src/Interpreters/ActionsVisitor.h | 27 --------------------------- 2 files changed, 18 insertions(+), 27 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 181ac9aed7e..3bad8fba270 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -252,6 +252,17 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co return header.cloneWithColumns(std::move(columns)); } + +namespace +{ + +/** Create a block for set from expression. + * 'set_element_types' - types of what are on the left hand side of IN. + * 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6). + * + * We need special implementation for ASTFunction, because in case, when we interpret + * large tuple or array as function, `evaluateConstantExpression` works extremely slow. + */ Block createBlockForSet( const DataTypePtr & left_arg_type, const ASTPtr & right_arg, @@ -295,6 +306,10 @@ Block createBlockForSet( return block; } +/** Create a block for set from literal. + * 'set_element_types' - types of what are on the left hand side of IN. + * 'right_arg' - Literal - Tuple or Array. + */ Block createBlockForSet( const DataTypePtr & left_arg_type, const std::shared_ptr & right_arg, @@ -346,6 +361,9 @@ Block createBlockForSet( return createBlockFromAST(elements_ast, set_element_types, context); } +} + + SetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, ContextPtr context, const SizeLimits & size_limits, PreparedSets & prepared_sets) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 342cc9eef9d..d1558cb961c 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -27,33 +27,6 @@ SetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, ContextPtr context, const SizeLimits & limits, PreparedSets & prepared_sets); -/** Create a block for set from expression. - * 'set_element_types' - types of what are on the left hand side of IN. - * 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6). - * - * We need special implementation for ASTFunction, because in case, when we interpret - * large tuple or array as function, `evaluateConstantExpression` works extremely slow. - * - * Note: this and following functions are used in third-party applications in Arcadia, so - * they should be declared in header file. - * - */ -Block createBlockForSet( - const DataTypePtr & left_arg_type, - const std::shared_ptr & right_arg, - const DataTypes & set_element_types, - ContextPtr context); - -/** Create a block for set from literal. - * 'set_element_types' - types of what are on the left hand side of IN. - * 'right_arg' - Literal - Tuple or Array. - */ -Block createBlockForSet( - const DataTypePtr & left_arg_type, - const ASTPtr & right_arg, - const DataTypes & set_element_types, - ContextPtr context); - /** For ActionsVisitor * A stack of ExpressionActions corresponding to nested lambda expressions. * The new action should be added to the highest possible level. From cbeeb7ec4f5d055344973d908c716e5d166ea0b3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 00:20:47 +0200 Subject: [PATCH 076/110] Remove Arcadia --- src/AggregateFunctions/UniquesHashSet.h | 2 +- src/Common/Config/ConfigProcessor.h | 3 +- src/Common/HashTable/Hash.h | 2 +- src/Common/OptimizedRegularExpression.cpp | 2 +- src/Common/StringUtils/CMakeLists.txt | 2 +- src/Common/ZooKeeper/ZooKeeperCommon.h | 2 +- src/Common/ZooKeeper/examples/CMakeLists.txt | 3 - .../examples/zk_many_watches_reconnect.cpp | 66 --------- src/Common/mysqlxx/mysqlxx/mysqlxx.h | 6 +- src/Common/parseAddress.h | 4 +- src/Functions/DateTimeTransforms.h | 2 +- src/Functions/FunctionFactory.h | 2 +- src/Functions/FunctionMathUnary.h | 2 +- src/Functions/FunctionsEmbeddedDictionaries.h | 2 +- src/Functions/FunctionsExternalDictionaries.h | 2 +- src/Functions/FunctionsVisitParam.h | 2 +- src/Functions/URL/FunctionsURL.h | 2 +- src/Functions/array/arrayEnumerateRanked.h | 2 +- src/Functions/array/arrayIndex.h | 4 +- src/Functions/extractTextFromHTML.cpp | 2 +- src/Functions/timeSlots.cpp | 2 +- src/IO/ReadHelpers.cpp | 2 +- src/IO/tests/gtest_s3_uri.cpp | 34 ++--- ...OptimizeIfWithConstantConditionVisitor.cpp | 2 +- src/Parsers/ASTFunction.cpp | 2 +- src/Parsers/ExpressionElementParsers.cpp | 2 +- .../Impl/JSONAsStringRowInputFormat.cpp | 1 + .../Algorithms/tests/gtest_graphite.cpp | 8 +- .../Transforms/ExpressionTransform.h | 2 +- src/Storages/examples/CMakeLists.txt | 13 -- .../examples/remove_symlink_directory.cpp | 35 ----- .../examples/transform_part_zk_nodes.cpp | 131 ------------------ 32 files changed, 49 insertions(+), 299 deletions(-) delete mode 100644 src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp delete mode 100644 src/Storages/examples/remove_symlink_directory.cpp delete mode 100644 src/Storages/examples/transform_part_zk_nodes.cpp diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index 6837803c67d..8648f6e2500 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -59,7 +59,7 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, - * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. + * stored in many places on disks (in many companies), so it continues to be used. */ struct UniquesHashSetDefaultHash { diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index 6c642690945..0e1d0facf48 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -55,7 +55,7 @@ public: /// 2) Determine the includes file from the config: /path2/metrika.xml /// If this path is not configured, use /etc/metrika.xml /// 3) Replace elements matching the "" pattern with - /// "contents of the yandex/bar element in metrika.xml" + /// "contents of the clickhouse/bar element in metrika.xml" /// 4) If zk_node_cache is non-NULL, replace elements matching the "" pattern with /// "contents of the /bar ZooKeeper node". /// If has_zk_includes is non-NULL and there are such elements, set has_zk_includes to true. @@ -137,4 +137,3 @@ private: }; } - diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index 5dbeeecf96b..3cf8978f418 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -13,7 +13,7 @@ * * Example: when we do aggregation by the visitor ID, the performance increase is more than 5 times. * This is because of following reasons: - * - in Yandex, visitor identifier is an integer that has timestamp with seconds resolution in lower bits; + * - in Metrica web analytics system, visitor identifier is an integer that has timestamp with seconds resolution in lower bits; * - in typical implementation of standard library, hash function for integers is trivial and just use lower bits; * - traffic is non-uniformly distributed across a day; * - we are using open-addressing linear probing hash tables that are most critical to hash function quality, diff --git a/src/Common/OptimizedRegularExpression.cpp b/src/Common/OptimizedRegularExpression.cpp index 1464923e6ab..da348adbe31 100644 --- a/src/Common/OptimizedRegularExpression.cpp +++ b/src/Common/OptimizedRegularExpression.cpp @@ -28,7 +28,7 @@ void OptimizedRegularExpressionImpl::analyze( * in which all metacharacters are escaped, * and also if there are no '|' outside the brackets, * and also avoid substrings of the form `http://` or `www` and some other - * (this is the hack for typical use case in Yandex.Metrica). + * (this is the hack for typical use case in web analytics applications). */ const char * begin = regexp.data(); const char * pos = begin; diff --git a/src/Common/StringUtils/CMakeLists.txt b/src/Common/StringUtils/CMakeLists.txt index 4eedbf4842b..57c196d335c 100644 --- a/src/Common/StringUtils/CMakeLists.txt +++ b/src/Common/StringUtils/CMakeLists.txt @@ -1,4 +1,4 @@ -# These files are located in separate library, because they are used by Yandex.Metrika code +# These files are located in separate library, because they are used by separate products # in places when no dependency on whole "dbms" library is possible. include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index e51bea3f7f8..532488c08f8 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -45,7 +45,7 @@ struct ZooKeeperResponse : virtual Response using ZooKeeperResponsePtr = std::shared_ptr; -/// Exposed in header file for Yandex.Metrica code. +/// Exposed in header file for some external code. struct ZooKeeperRequest : virtual Request { XID xid = 0; diff --git a/src/Common/ZooKeeper/examples/CMakeLists.txt b/src/Common/ZooKeeper/examples/CMakeLists.txt index 8bec951e24f..b449b172605 100644 --- a/src/Common/ZooKeeper/examples/CMakeLists.txt +++ b/src/Common/ZooKeeper/examples/CMakeLists.txt @@ -7,8 +7,5 @@ target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zoo add_executable(zkutil_test_async zkutil_test_async.cpp) target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper_no_log) -add_executable (zk_many_watches_reconnect zk_many_watches_reconnect.cpp) -target_link_libraries (zk_many_watches_reconnect PRIVATE clickhouse_common_zookeeper_no_log clickhouse_common_config) - add_executable (zookeeper_impl zookeeper_impl.cpp) target_link_libraries (zookeeper_impl PRIVATE clickhouse_common_zookeeper_no_log) diff --git a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp b/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp deleted file mode 100644 index cf819121234..00000000000 --- a/src/Common/ZooKeeper/examples/zk_many_watches_reconnect.cpp +++ /dev/null @@ -1,66 +0,0 @@ -#include -#include -#include -#include - -/// A tool for reproducing https://issues.apache.org/jira/browse/ZOOKEEPER-706 -/// Original libzookeeper can't reconnect the session if the length of SET_WATCHES message -/// exceeds jute.maxbuffer (0xfffff by default). -/// This happens when the number of watches exceeds ~29000. -/// -/// Session reconnect can be caused by forbidding packets to the current zookeeper server, e.g. -/// sudo ip6tables -A OUTPUT -d mtzoo01it.haze.yandex.net -j REJECT - -const size_t N_THREADS = 100; - -int main(int argc, char ** argv) -{ - try - { - if (argc != 3) - { - std::cerr << "usage: " << argv[0] << " " << std::endl; - return 3; - } - - DB::ConfigProcessor processor(argv[1], false, true); - auto config = processor.loadConfig().configuration; - zkutil::ZooKeeper zk(*config, "zookeeper", nullptr); - zkutil::EventPtr watch = std::make_shared(); - - /// NOTE: setting watches in multiple threads because doing it in a single thread is too slow. - size_t watches_per_thread = std::stoull(argv[2]) / N_THREADS; - std::vector threads; - for (size_t i_thread = 0; i_thread < N_THREADS; ++i_thread) - { - threads.emplace_back([&, i_thread] - { - for (size_t i = 0; i < watches_per_thread; ++i) - zk.exists("/clickhouse/nonexistent_node" + std::to_string(i * N_THREADS + i_thread), nullptr, watch); - }); - } - for (size_t i_thread = 0; i_thread < N_THREADS; ++i_thread) - threads[i_thread].join(); - - while (true) - { - std::cerr << "WAITING..." << std::endl; - sleep(10); - } - } - catch (Poco::Exception & e) - { - std::cerr << "Exception: " << e.displayText() << std::endl; - return 1; - } - catch (std::exception & e) - { - std::cerr << "std::exception: " << e.what() << std::endl; - return 3; - } - catch (...) - { - std::cerr << "Some exception" << std::endl; - return 2; - } -} diff --git a/src/Common/mysqlxx/mysqlxx/mysqlxx.h b/src/Common/mysqlxx/mysqlxx/mysqlxx.h index 785d4361fd7..0b6cf5e7c1a 100644 --- a/src/Common/mysqlxx/mysqlxx/mysqlxx.h +++ b/src/Common/mysqlxx/mysqlxx/mysqlxx.h @@ -10,10 +10,10 @@ /** 'mysqlxx' - very simple library for replacement of 'mysql++' library. * - * For whatever reason, in Yandex.Metrica, back in 2008, 'mysql++' library was used. + * For whatever reason, in Metrica web analytics system, back in 2008, 'mysql++' library was used. * There are the following shortcomings of 'mysql++': * 1. Too rich functionality: most of it is not used. - * 2. Low performance (when used for Yandex.Metrica). + * 2. Low performance (when used for Metrica). * * Low performance is caused by the following reasons: * @@ -50,7 +50,7 @@ * And for the sake of simplicity, some functions work only with certain assumptions, * or with slightly different semantic than in mysql++. * And we don't care about cross-platform usage of mysqlxx. - * These assumptions are specific for Yandex.Metrica. Your mileage may vary. + * These assumptions are specific for Metrica. Your mileage may vary. * * mysqlxx could not be considered as separate full-featured library, * because it is developed from the principle - "everything that we don't need is not implemented". diff --git a/src/Common/parseAddress.h b/src/Common/parseAddress.h index 86340982c67..297934a6379 100644 --- a/src/Common/parseAddress.h +++ b/src/Common/parseAddress.h @@ -13,8 +13,8 @@ namespace DB * Otherwise, an exception is thrown. * * Examples: - * yandex.ru - returns "yandex.ru" and default_port - * yandex.ru:80 - returns "yandex.ru" and 80 + * clickhouse.com - returns "clickhouse.com" and default_port + * clickhouse.com:80 - returns "clickhouse.com" and 80 * [2a02:6b8:a::a]:80 - returns [2a02:6b8:a::a] and 80; note that square brackets remain in returned host. */ std::pair parseAddress(const std::string & str, UInt16 default_port); diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 3a7baf45adc..5d1bcaf48cf 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -539,7 +539,7 @@ struct ToStartOfFifteenMinutesImpl using FactorTransform = ZeroTransform; }; -/// Round to start of half-an-hour length interval with unspecified offset. This transform is specific for Yandex.Metrica. +/// Round to start of half-an-hour length interval with unspecified offset. This transform is specific for Metrica web analytics system. struct TimeSlotImpl { static constexpr auto name = "timeSlot"; diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index 898a463cd58..13b14559ec4 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -40,7 +40,7 @@ public: registerFunction(name, &Function::create, case_sensitiveness); } - /// This function is used by YQL - internal Yandex product that depends on ClickHouse by source code. + /// This function is used by YQL - innovative transactional DBMS that depends on ClickHouse by source code. std::vector getAllNames() const; bool has(const std::string & name) const; diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index fa10c004e87..bd656db792b 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -10,7 +10,7 @@ #include "config_functions.h" -/** FastOps is a fast vector math library from Mikhail Parakhin (former Yandex CTO), +/** FastOps is a fast vector math library from Mikhail Parakhin, https://www.linkedin.com/in/mikhail-parakhin/ * Enabled by default. */ #if USE_FASTOPS diff --git a/src/Functions/FunctionsEmbeddedDictionaries.h b/src/Functions/FunctionsEmbeddedDictionaries.h index c6ea886b4a8..20be3ee3cce 100644 --- a/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/src/Functions/FunctionsEmbeddedDictionaries.h @@ -33,7 +33,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -/** Functions using Yandex.Metrica dictionaries +/** Functions using deprecated dictionaries * - dictionaries of regions, operating systems, search engines. * * Climb up the tree to a certain level. diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 6a701d7b864..189ec7321c1 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -195,7 +195,7 @@ public: * that is the initiator of a distributed query, * in the case when the function will be invoked for real data only at the remote servers. * This feature is controversial and implemented specially - * for backward compatibility with the case in Yandex Banner System. + * for backward compatibility with the case in the Banner System application. */ if (input_rows_count == 0) return result_type->createColumn(); diff --git a/src/Functions/FunctionsVisitParam.h b/src/Functions/FunctionsVisitParam.h index 362c3bcd693..09fcf8659ed 100644 --- a/src/Functions/FunctionsVisitParam.h +++ b/src/Functions/FunctionsVisitParam.h @@ -12,7 +12,7 @@ /** Functions for retrieving "visit parameters". - * Visit parameters in Yandex.Metrika are a special kind of JSONs. + * Visit parameters in Metrica web analytics system are a special kind of JSONs. * These functions are applicable to almost any JSONs. * Implemented via templates from FunctionsStringSearch.h. * diff --git a/src/Functions/URL/FunctionsURL.h b/src/Functions/URL/FunctionsURL.h index 297b62ca256..a0f106742fb 100644 --- a/src/Functions/URL/FunctionsURL.h +++ b/src/Functions/URL/FunctionsURL.h @@ -45,7 +45,7 @@ namespace DB * Remove specified parameter from URL. * cutURLParameter(URL, name) * - * Get array of URL 'hierarchy' as in Yandex.Metrica tree-like reports. See docs. + * Get array of URL 'hierarchy' as in web-analytics tree-like reports. See the docs. * URLHierarchy(URL) */ diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index d6a62a966ae..ac3936af0fd 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -15,7 +15,7 @@ // for better debug: #include /** The function will enumerate distinct values of the passed multidimensional arrays looking inside at the specified depths. - * This is very unusual function made as a special order for Yandex.Metrica. + * This is very unusual function made as a special order for our dear customer - Metrica web analytics system. * * arrayEnumerateUniqRanked(['hello', 'world', 'hello']) = [1, 1, 2] * - it returns similar structured array containing number of occurrence of the corresponding value. diff --git a/src/Functions/array/arrayIndex.h b/src/Functions/array/arrayIndex.h index 8b42b99cd69..0dbbe5e41b6 100644 --- a/src/Functions/array/arrayIndex.h +++ b/src/Functions/array/arrayIndex.h @@ -718,9 +718,7 @@ private: /** * Catches arguments of type LowCardinality(T) (left) and U (right). * - * The perftests - * https://clickhouse-test-reports.s3.yandex.net/12550/2d27fa0fa8c198a82bf1fe3625050ccf56695976/integration_tests_(release).html - * showed that the amount of action needed to convert the non-constant right argument to the index column + * The perftests showed that the amount of action needed to convert the non-constant right argument to the index column * (similar to the left one's) is significantly higher than converting the array itself to an ordinary column. * * So, in terms of performance it's more optimal to fall back to default implementation and catch only constant diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index b38ea74d6ce..f321a59f734 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -53,7 +53,7 @@ * * Usage example: * - * SELECT extractTextFromHTML(html) FROM url('https://yandex.ru/', RawBLOB, 'html String') + * SELECT extractTextFromHTML(html) FROM url('https://github.com/ClickHouse/ClickHouse', RawBLOB, 'html String') * * - ClickHouse has embedded web browser. */ diff --git a/src/Functions/timeSlots.cpp b/src/Functions/timeSlots.cpp index 34af410befa..a19ccf62565 100644 --- a/src/Functions/timeSlots.cpp +++ b/src/Functions/timeSlots.cpp @@ -29,7 +29,7 @@ namespace * For example, timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]. * This is necessary to search for hits that are part of the corresponding visit. * - * This is obsolete function. It was developed for Yandex.Metrica, but no longer used in Yandex. + * This is obsolete function. It was developed for Metrica web analytics system, but the art of its usage has been forgotten. * But this function was adopted by wider audience. */ diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index bf3cccccab8..8aee5713cdc 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -343,7 +343,7 @@ static void parseComplexEscapeSequence(Vector & s, ReadBuffer & buf) && decoded_char != '"' && decoded_char != '`' /// MySQL style identifiers && decoded_char != '/' /// JavaScript in HTML - && decoded_char != '=' /// Yandex's TSKV + && decoded_char != '=' /// TSKV format invented somewhere && !isControlASCII(decoded_char)) { s.push_back('\\'); diff --git a/src/IO/tests/gtest_s3_uri.cpp b/src/IO/tests/gtest_s3_uri.cpp index 7ee72069e57..20d19437c64 100644 --- a/src/IO/tests/gtest_s3_uri.cpp +++ b/src/IO/tests/gtest_s3_uri.cpp @@ -16,36 +16,36 @@ class S3UriTest : public testing::TestWithParam TEST(S3UriTest, validPatterns) { { - S3::URI uri(Poco::URI("https://jokserfn.s3.yandexcloud.net/")); - ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://jokserfn.s3.amazonaws.com/")); + ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("", uri.key); ASSERT_EQ(true, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://s3.yandexcloud.net/jokserfn/")); - ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://s3.amazonaws.com/jokserfn/")); + ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("", uri.key); ASSERT_EQ(false, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://yandexcloud.net/bucket/")); - ASSERT_EQ("https://yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://amazonaws.com/bucket/")); + ASSERT_EQ("https://amazonaws.com", uri.endpoint); ASSERT_EQ("bucket", uri.bucket); ASSERT_EQ("", uri.key); ASSERT_EQ(false, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://jokserfn.s3.yandexcloud.net/data")); - ASSERT_EQ("https://s3.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://jokserfn.s3.amazonaws.com/data")); + ASSERT_EQ("https://s3.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("data", uri.key); ASSERT_EQ(true, uri.is_virtual_hosted_style); } { - S3::URI uri(Poco::URI("https://storage.yandexcloud.net/jokserfn/data")); - ASSERT_EQ("https://storage.yandexcloud.net", uri.endpoint); + S3::URI uri(Poco::URI("https://storage.amazonaws.com/jokserfn/data")); + ASSERT_EQ("https://storage.amazonaws.com", uri.endpoint); ASSERT_EQ("jokserfn", uri.bucket); ASSERT_EQ("data", uri.key); ASSERT_EQ(false, uri.is_virtual_hosted_style); @@ -97,13 +97,13 @@ INSTANTIATE_TEST_SUITE_P( S3UriTest, testing::Values( "https:///", - "https://.s3.yandexcloud.net/key", - "https://s3.yandexcloud.net/key", - "https://jokserfn.s3yandexcloud.net/key", - "https://s3.yandexcloud.net//", - "https://yandexcloud.net/", - "https://yandexcloud.net//", - "https://yandexcloud.net//key")); + "https://.s3.amazonaws.com/key", + "https://s3.amazonaws.com/key", + "https://jokserfn.s3amazonaws.com/key", + "https://s3.amazonaws.com//", + "https://amazonaws.com/", + "https://amazonaws.com//", + "https://amazonaws.com//key")); } diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 802bf4e43ce..28b19f6670d 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -28,7 +28,7 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v } /// cast of numeric constant in condition to UInt8 - /// Note: this solution is ad-hoc and only implemented for yandex.metrica use case. + /// Note: this solution is ad-hoc and only implemented for metrica use case (one of the best customers). /// We should allow any constant condition (or maybe remove this optimization completely) later. if (const auto * function = condition->as()) { diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index f9a5c7be75f..b86929b054c 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -166,7 +166,7 @@ ASTPtr ASTFunction::toLiteral() const * Motivation: most people are unaware that _ is a metacharacter and forgot to properly escape it with two backslashes. * With highlighting we make it clearly obvious. * - * Another case is regexp match. Suppose the user types match(URL, 'www.yandex.ru'). It often means that the user is unaware that . is a metacharacter. + * Another case is regexp match. Suppose the user types match(URL, 'www.clickhouse.com'). It often means that the user is unaware that . is a metacharacter. */ static bool highlightStringLiteralWithMetacharacters(const ASTPtr & node, const IAST::FormatSettings & settings, const char * metacharacters) { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index cd07e304a39..29c7846283e 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1727,7 +1727,7 @@ const char * ParserAlias::restricted_keywords[] = "NOT", "OFFSET", "ON", - "ONLY", /// YQL synonym for ANTI. Note: YQL is the name of one of Yandex proprietary languages, completely unrelated to ClickHouse. + "ONLY", /// YQL's synonym for ANTI. Note: YQL is the name of one of proprietary languages, completely unrelated to ClickHouse. "ORDER", "PREWHERE", "RIGHT", diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index 9bf1682b77e..e31006ff0f6 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -52,6 +52,7 @@ void JSONAsRowInputFormat::readSuffix() { assertChar(']', *buf); skipWhitespaceIfAny(*buf); + data_in_square_brackets = false; } if (!buf->eof() && *buf->position() == ';') { diff --git a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp index 1d739bf566a..9160ad6e0fa 100644 --- a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp +++ b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp @@ -149,7 +149,7 @@ TEST(GraphiteTest, testSelectPattern) using namespace std::literals; std::string - xml(R"END( + xml(R"END( \.sum$ @@ -210,7 +210,7 @@ TEST(GraphiteTest, testSelectPattern) - + )END"); // Retentions must be ordered by 'age' descending. @@ -370,7 +370,7 @@ TEST(GraphiteTest, testSelectPatternTyped) using namespace std::literals; std::string - xml(R"END( + xml(R"END( plain @@ -488,7 +488,7 @@ TEST(GraphiteTest, testSelectPatternTyped) - + )END"); // Retentions must be ordered by 'age' descending. diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index ea73c8fb1da..791c7d7ba73 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -12,7 +12,7 @@ class ActionsDAG; /** Executes a certain expression over the block. * The expression consists of column identifiers from the block, constants, common functions. - * For example: hits * 2 + 3, url LIKE '%yandex%' + * For example: hits * 2 + 3, url LIKE '%clickhouse%' * The expression processes each row independently of the others. */ class ExpressionTransform final : public ISimpleTransform diff --git a/src/Storages/examples/CMakeLists.txt b/src/Storages/examples/CMakeLists.txt index 103972a106f..ca85ca9d98f 100644 --- a/src/Storages/examples/CMakeLists.txt +++ b/src/Storages/examples/CMakeLists.txt @@ -1,6 +1,3 @@ -add_executable (remove_symlink_directory remove_symlink_directory.cpp) -target_link_libraries (remove_symlink_directory PRIVATE dbms) - add_executable (merge_selector merge_selector.cpp) target_link_libraries (merge_selector PRIVATE dbms) @@ -12,13 +9,3 @@ target_link_libraries (get_current_inserts_in_replicated PRIVATE dbms clickhouse add_executable (get_abandonable_lock_in_all_partitions get_abandonable_lock_in_all_partitions.cpp) target_link_libraries (get_abandonable_lock_in_all_partitions PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper) - -add_executable (transform_part_zk_nodes transform_part_zk_nodes.cpp) -target_link_libraries (transform_part_zk_nodes - PRIVATE - boost::program_options - clickhouse_common_config - clickhouse_common_zookeeper - dbms - string_utils -) diff --git a/src/Storages/examples/remove_symlink_directory.cpp b/src/Storages/examples/remove_symlink_directory.cpp deleted file mode 100644 index db436c0a608..00000000000 --- a/src/Storages/examples/remove_symlink_directory.cpp +++ /dev/null @@ -1,35 +0,0 @@ -#include -#include -#include -#include -#include - -namespace fs = std::filesystem; - -namespace DB -{ - namespace ErrorCodes - { - extern const int SYSTEM_ERROR; - } -} - -int main(int, char **) -try -{ - fs::path dir("./test_dir/"); - fs::create_directories(dir); - FS::createFile("./test_dir/file"); - - if (0 != symlink("./test_dir", "./test_link")) - DB::throwFromErrnoWithPath("Cannot create symlink", "./test_link", DB::ErrorCodes::SYSTEM_ERROR); - - fs::rename("./test_link", "./test_link2"); - fs::remove_all("./test_link2"); - return 0; -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(false) << "\n"; - return 1; -} diff --git a/src/Storages/examples/transform_part_zk_nodes.cpp b/src/Storages/examples/transform_part_zk_nodes.cpp deleted file mode 100644 index 3cbcc76190e..00000000000 --- a/src/Storages/examples/transform_part_zk_nodes.cpp +++ /dev/null @@ -1,131 +0,0 @@ -#include -#include -#include -#include -#include - -#include - -#include -#include - - -int main(int argc, char ** argv) -try -{ - boost::program_options::options_description desc("Allowed options"); - desc.add_options() - ("help,h", "produce help message") - ("address,a", boost::program_options::value()->required(), - "addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181") - ("path,p", boost::program_options::value()->required(), - "where to start") - ; - - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Transform contents of part nodes in ZooKeeper to more compact storage scheme." << std::endl; - std::cout << "Usage: " << argv[0] << " [options]" << std::endl; - std::cout << desc << std::endl; - return 1; - } - - zkutil::ZooKeeper zookeeper(options.at("address").as()); - - std::string initial_path = options.at("path").as(); - - struct Node - { - Node( - std::string path_, - std::future get_future_, - std::future children_future_, - Node * parent_) - : path(std::move(path_)) - , get_future(std::move(get_future_)) - , children_future(std::move(children_future_)) - , parent(parent_) - { - } - - std::string path; - std::future get_future; - std::future children_future; - - Node * parent = nullptr; - std::future set_future; - }; - - std::list nodes_queue; - nodes_queue.emplace_back( - initial_path, zookeeper.asyncGet(initial_path), zookeeper.asyncGetChildren(initial_path), nullptr); - - for (auto it = nodes_queue.begin(); it != nodes_queue.end(); ++it) - { - Coordination::GetResponse get_response; - Coordination::ListResponse children_response; - try - { - get_response = it->get_future.get(); - children_response = it->children_future.get(); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::Error::ZNONODE) - continue; - throw; - } - - if (get_response.stat.ephemeralOwner) - continue; - - if (it->path.find("/parts/") != std::string::npos - && !endsWith(it->path, "/columns") - && !endsWith(it->path, "/checksums")) - { - /// The node is related to part. - - /// If it is the part in old format (the node contains children) - convert it to the new format. - if (!children_response.names.empty()) - { - auto part_header = DB::ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes( - zookeeper.get(it->path + "/columns"), zookeeper.get(it->path + "/checksums")); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeRemoveRequest(it->path + "/columns", -1)); - ops.emplace_back(zkutil::makeRemoveRequest(it->path + "/checksums", -1)); - ops.emplace_back(zkutil::makeSetRequest(it->path, part_header.toString(), -1)); - - it->set_future = zookeeper.asyncMulti(ops); - } - } - else - { - /// Recursively add children to the queue. - for (const auto & name : children_response.names) - { - std::string child_path = it->path == "/" ? it->path + name : it->path + '/' + name; - nodes_queue.emplace_back( - child_path, zookeeper.asyncGet(child_path), zookeeper.asyncGetChildren(child_path), - &(*it)); - } - } - } - - for (auto & node : nodes_queue) - { - if (node.set_future.valid()) - { - node.set_future.get(); - std::cerr << node.path << " changed!" << std::endl; - } - } -} -catch (...) -{ - std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; - throw; -} From e128d89957816877944e37531a4e9acc0ed477b5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 00:24:58 +0200 Subject: [PATCH 077/110] Remove Arcadia --- src/Disks/DiskWebServer.h | 2 +- .../clickhouse-diagnostics | 2 +- utils/graphite-rollup/rollup-tag-list.xml | 4 +- utils/graphite-rollup/rollup-typed.xml | 4 +- utils/graphite-rollup/rollup.xml | 4 +- utils/release/push_packages | 250 ------------------ utils/s3tools/s3uploader | 4 +- .../main.cpp | 2 +- .../main.cpp | 2 +- utils/zookeeper-dump-tree/main.cpp | 2 +- utils/zookeeper-remove-by-list/main.cpp | 2 +- 11 files changed, 14 insertions(+), 264 deletions(-) delete mode 100755 utils/release/push_packages diff --git a/src/Disks/DiskWebServer.h b/src/Disks/DiskWebServer.h index 6341b582174..98f92fe5986 100644 --- a/src/Disks/DiskWebServer.h +++ b/src/Disks/DiskWebServer.h @@ -20,7 +20,7 @@ namespace ErrorCodes * * * web - * https://clickhouse-datasets.s3.yandex.net/disk-with-static-files-tests/test-hits/ + * https://clickhouse-datasets.s3.amazonaws.com/disk-with-static-files-tests/test-hits/ * * * diff --git a/utils/clickhouse-diagnostics/clickhouse-diagnostics b/utils/clickhouse-diagnostics/clickhouse-diagnostics index 2fe67071c3c..cf65e4efbfb 100755 --- a/utils/clickhouse-diagnostics/clickhouse-diagnostics +++ b/utils/clickhouse-diagnostics/clickhouse-diagnostics @@ -665,7 +665,7 @@ class DiagnosticsData: def _dump_wiki(self): """ - Dump diagnostic data in Yandex wiki format. + Dump diagnostic data in Wiki format. """ def _write_title(buffer, value): diff --git a/utils/graphite-rollup/rollup-tag-list.xml b/utils/graphite-rollup/rollup-tag-list.xml index ef28f2089ad..edab2f16436 100644 --- a/utils/graphite-rollup/rollup-tag-list.xml +++ b/utils/graphite-rollup/rollup-tag-list.xml @@ -1,4 +1,4 @@ - + plain @@ -164,4 +164,4 @@ - + diff --git a/utils/graphite-rollup/rollup-typed.xml b/utils/graphite-rollup/rollup-typed.xml index 0b27d43ece9..ace439dba4a 100644 --- a/utils/graphite-rollup/rollup-typed.xml +++ b/utils/graphite-rollup/rollup-typed.xml @@ -1,4 +1,4 @@ - + plain @@ -164,4 +164,4 @@ - + diff --git a/utils/graphite-rollup/rollup.xml b/utils/graphite-rollup/rollup.xml index 641b0130509..2089605c8bf 100644 --- a/utils/graphite-rollup/rollup.xml +++ b/utils/graphite-rollup/rollup.xml @@ -1,4 +1,4 @@ - + \.sum$ @@ -144,4 +144,4 @@ - + diff --git a/utils/release/push_packages b/utils/release/push_packages deleted file mode 100755 index e25cb325c71..00000000000 --- a/utils/release/push_packages +++ /dev/null @@ -1,250 +0,0 @@ -#!/usr/bin/env python3 -# -*- coding: utf-8 -*- -import argparse -import subprocess -import os -import logging -import shutil -import base64 -import pexpect - - -# Do nothing if keys are not provided -class GpgKey(object): - gnupg_dir = os.path.expanduser('~/.gnupg') - TEMPGNUPG_DIR = os.path.expanduser('~/.local/tempgnupg') - - def __init__(self, secret_key_path, public_key_path): - if secret_key_path and public_key_path: - with open(secret_key_path, 'r') as sec, open(public_key_path, 'r') as pub: - self._secret_key = sec.read() - self._public_key = pub.read() - else: - self._secret_key = None - self._public_key = None - - def __enter__(self): - if self._secret_key and self._public_key: - if os.path.exists(self.gnupg_dir): - shutil.move(self.gnupg_dir, self.TEMPGNUPG_DIR) - os.mkdir(self.gnupg_dir) - open(os.path.join(self.gnupg_dir, 'secring.gpg'), 'wb').write(base64.b64decode(self._secret_key)) - open(os.path.join(self.gnupg_dir, 'pubring.gpg'), 'wb').write(base64.b64decode(self._public_key)) - return self - - def __exit__(self, exc_type, exc_val, exc_tb): - if self._secret_key and self._public_key: - shutil.rmtree(self.gnupg_dir) - if os.path.exists(self.TEMPGNUPG_DIR): - shutil.move(self.TEMPGNUPG_DIR, self.gnupg_dir) - - -class DebRelease(object): - - DUPLOAD_CONF_TEMPLATE = '\n\t'.join(( - "$cfg{{'{title}'}} = {{", - 'fqdn => "{fqdn}",', - 'method => "{method}",', - 'login => "{login}",', - 'incoming => "{incoming}",', - 'options => "{options}",', - 'dinstall_runs => {dinstall_runs},\n}};',)) - DUPLOAD_CONF_PATH = os.path.expanduser('~/.dupload.conf') - DUPLOAD_CONF_TMP_PATH = os.path.expanduser('~/.local/tmp_dupload.cnf') - - def __init__(self, dupload_config, login, ssh_key_path): - self.__config = {} - for repo, conf in dupload_config.items(): - d = { - "fqdn": conf["fqdn"], - "method": "scpb", - "login": login, - "incoming": conf["incoming"], - "dinstall_runs": 0, - "options": "-o StrictHostKeyChecking=no -o UserKnownHostsFile=/dev/null -o ConnectionAttempts=3", - } - d.update(conf) - self.__config[repo] = d - print(self.__config) - self.ssh_key_path = ssh_key_path - - def __enter__(self): - if os.path.exists(self.DUPLOAD_CONF_PATH): - shutil.move(self.DUPLOAD_CONF_PATH, self.DUPLOAD_CONF_TMP_PATH) - self.__dupload_conf = open(self.DUPLOAD_CONF_PATH, 'w') - self.__dupload_conf.write('package config;\n\n$default_host = undef;\n\n' + '\n\n'.join([ - self.DUPLOAD_CONF_TEMPLATE.format(title=title, **values) - for title, values in self.__config.items()])) - self.__dupload_conf.write('\n') - self.__dupload_conf.close() - if self.ssh_key_path: - subprocess.check_call("ssh-add {}".format(self.ssh_key_path), shell=True) - return self - - def __exit__(self, exc_type, exc_val, exc_tb): - if os.path.exists(self.DUPLOAD_CONF_TMP_PATH): - shutil.move(self.DUPLOAD_CONF_TMP_PATH, self.DUPLOAD_CONF_PATH) - else: - os.unlink(self.DUPLOAD_CONF_PATH) - - -class SSHConnection(object): - def __init__(self, user, host, ssh_key=None): - if ssh_key: - key_str = "-i {}".format(ssh_key) - else: - key_str = "" - - self.base_cmd = "ssh {key} {user}@{host}".format( - key=key_str, user=user, host=host) - - def execute(self, cmd): - logging.info("Executing remote cmd %s", cmd) - subprocess.check_call(self.base_cmd + ' "{cmd}"'.format(cmd=cmd), - shell=True) - - -def debsign(path, gpg_passphrase, gpg_sec_key_path, gpg_pub_key_path, gpg_user): - try: - with GpgKey(gpg_sec_key_path, gpg_pub_key_path): - cmd = ('debsign -k \'{key}\' -p"gpg --verbose --no-use-agent --batch ' - '--no-tty --passphrase {passphrase}" {path}/*.changes').format( - key=gpg_user, passphrase=gpg_passphrase, path=path) - logging.info("Build debsign cmd '%s'", cmd) - subprocess.check_call(cmd, shell=True) - logging.info("debsign finished") - except Exception as ex: - logging.error("Cannot debsign packages on path %s, with user key", path) - raise ex - -def rpmsign(path, gpg_passphrase, gpg_sec_key_path, gpg_pub_key_path, gpg_user): - try: - with GpgKey(gpg_sec_key_path, gpg_pub_key_path): - for package in os.listdir(path): - package_path = os.path.join(path, package) - logging.info("Signing %s", package_path) - proc = pexpect.spawn('rpm --resign -D "_signature gpg" -D "_gpg_name {username}" {package}'.format(username=gpg_user, package=package_path)) - proc.expect_exact("Enter pass phrase: ") - proc.sendline(gpg_passphrase) - proc.expect(pexpect.EOF) - logging.info("Signed successfully") - except Exception as ex: - logging.error("Cannot rpmsign packages on path %s, with user key", path) - raise ex - -def transfer_packages_scp(ssh_key, path, repo_user, repo_url, incoming_directory): - logging.info("Transferring packages via scp to %s", repo_url) - if ssh_key: - key_str = "-i {}".format(ssh_key) - else: - key_str = "" - subprocess.check_call('scp {key_str} {path}/* {user}@{repo}:{incoming}'.format( - path=path, user=repo_user, repo=repo_url, key_str=key_str, incoming=incoming_directory), shell=True) - logging.info("Transfer via scp finished") - -def transfer_packages_dupload(ssh_key, path, repo_user, repo_url, incoming_directory): - repo_short_name = repo_url.split('.')[0] - config = { - repo_short_name: { - "fqdn": repo_url, - "incoming": incoming_directory, - } - } - with DebRelease(config, repo_user, ssh_key): - logging.info("Duploading") - subprocess.check_call("dupload -f --nomail --to {repo} {path}".format(repo=repo_short_name, path=path), shell=True) - logging.info("Dupload finished") - - -def clear_old_incoming_packages(ssh_connection, user): - for pkg in ('deb', 'rpm', 'tgz'): - for release_type in ('stable', 'testing', 'prestable', 'lts'): - try: - ssh_connection.execute("rm /home/{user}/incoming/clickhouse/{pkg}/{release_type}/*".format( - user=user, pkg=pkg, release_type=release_type)) - except Exception: - logging.info("rm is not required") - - -def _get_incoming_path(repo_url, user=None, pkg_type=None, release_type=None): - if repo_url == 'repo.mirror.yandex.net': - return "/home/{user}/incoming/clickhouse/{pkg}/{release_type}".format( - user=user, pkg=pkg_type, release_type=release_type) - else: - return "/repo/{0}/mini-dinstall/incoming/".format(repo_url.split('.')[0]) - - -def _fix_args(args): - - if args.gpg_sec_key_path and not os.path.isabs(args.gpg_sec_key_path): - args.gpg_sec_key_path = os.path.join(os.getcwd(), args.gpg_sec_key_path) - - if args.gpg_pub_key_path and not os.path.isabs(args.gpg_pub_key_path): - args.gpg_pub_key_path = os.path.join(os.getcwd(), args.gpg_pub_key_path) - - if args.ssh_key_path and not os.path.isabs(args.ssh_key_path): - args.ssh_key_path = os.path.join(os.getcwd(), args.ssh_key_path) - - -if __name__ == "__main__": - logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') - parser = argparse.ArgumentParser(description="Program to push clickhouse packages to repository") - parser.add_argument('--deb-directory') - parser.add_argument('--rpm-directory') - parser.add_argument('--tgz-directory') - parser.add_argument('--release-type', choices=('testing', 'stable', 'prestable', 'lts'), default='testing') - parser.add_argument('--ssh-key-path') - parser.add_argument('--gpg-passphrase', required=True) - parser.add_argument('--gpg-sec-key-path') - parser.add_argument('--gpg-pub-key-path') - parser.add_argument('--gpg-key-user', default='robot-clickhouse') - parser.add_argument('--repo-url', default='repo.mirror.yandex.net') - parser.add_argument('--repo-user', default='buildfarm') - - args = parser.parse_args() - if args.deb_directory is None and args.rpm_directory is None and args.tgz_directory is None: - parser.error('At least one package directory required') - - _fix_args(args) - - is_open_source = args.repo_url == 'repo.mirror.yandex.net' - ssh_connection = SSHConnection(args.repo_user, args.repo_url, args.ssh_key_path) - - packages = [] - if args.deb_directory: - debsign(args.deb_directory, args.gpg_passphrase, args.gpg_sec_key_path, args.gpg_pub_key_path, args.gpg_key_user) - packages.append((args.deb_directory, 'deb')) - - if args.rpm_directory: - if not is_open_source: - raise Exception("Cannot upload .rpm package to {}".format(args.repo_url)) - rpmsign(args.rpm_directory, args.gpg_passphrase, args.gpg_sec_key_path, args.gpg_pub_key_path, args.gpg_key_user) - packages.append((args.rpm_directory, 'rpm')) - - if args.tgz_directory: - if not is_open_source: - raise Exception("Cannot upload .tgz package to {}".format(args.repo_url)) - packages.append((args.tgz_directory, 'tgz')) - - if is_open_source: - logging.info("Clearing old directory with incoming packages on buildfarm") - clear_old_incoming_packages(ssh_connection, args.repo_user) - logging.info("Incoming directory cleared") - - for package_path, package_type in packages: - logging.info("Processing path '%s' with package type %s", package_path, package_type) - incoming_directory = _get_incoming_path(args.repo_url, args.repo_user, package_type, args.release_type) - if package_type == "deb": - transfer_packages_dupload(args.ssh_key_path, package_path, args.repo_user, args.repo_url, incoming_directory) - else: - transfer_packages_scp(args.ssh_key_path, package_path, args.repo_user, args.repo_url, incoming_directory) - - logging.info("Running clickhouse install (it takes about (20-30 minutes)") - ssh_connection.execute("sudo /usr/sbin/ya-clickhouse-{0}-install".format(package_type)) - logging.info("Clickhouse installed") - logging.info("Pushing clickhouse to repo") - ssh_connection.execute("/usr/sbin/push2publicrepo.sh clickhouse") - logging.info("Push finished") - logging.info("Package '%s' pushed", package_type) - else: - transfer_packages_dupload(args.ssh_key_path, args.deb_directory, args.repo_user, args.repo_url, _get_incoming_path(args.repo_url)) diff --git a/utils/s3tools/s3uploader b/utils/s3tools/s3uploader index 0627b19ea8f..33db76f57f4 100755 --- a/utils/s3tools/s3uploader +++ b/utils/s3tools/s3uploader @@ -121,8 +121,8 @@ if __name__ == "__main__": parser = argparse.ArgumentParser( description="Simple tool for uploading datasets to clickhouse S3", usage='%(prog)s [options] {}'.format(USAGE_EXAMPLES)) - parser.add_argument('--s3-api-url', default='s3.mds.yandex.net') - parser.add_argument('--s3-common-url', default='s3.yandex.net') + parser.add_argument('--s3-api-url', default='s3.amazonaws.com') + parser.add_argument('--s3-common-url', default='s3.amazonaws.com') parser.add_argument('--bucket-name', default='clickhouse-datasets') parser.add_argument('--dataset-name', required=True, help='Name of dataset, will be used in uploaded path') diff --git a/utils/zookeeper-adjust-block-numbers-to-parts/main.cpp b/utils/zookeeper-adjust-block-numbers-to-parts/main.cpp index 8550675cb9e..5c694ee04ef 100644 --- a/utils/zookeeper-adjust-block-numbers-to-parts/main.cpp +++ b/utils/zookeeper-adjust-block-numbers-to-parts/main.cpp @@ -214,7 +214,7 @@ try po::options_description desc("Allowed options"); desc.add_options() ("help,h", "show help") - ("zookeeper,z", po::value(), "Addresses of ZooKeeper instances, comma-separated. Example: example01e.yandex.ru:2181") + ("zookeeper,z", po::value(), "Addresses of ZooKeeper instances, comma-separated. Example: example01e.clickhouse.com:2181") ("path,p", po::value(), "[optional] Path of replica queue to insert node (without trailing slash). By default it's /clickhouse/tables") ("shard,s", po::value(), "[optional] Shards to process, comma-separated. If not specified then the utility will process all the shards.") ("table,t", po::value(), "[optional] Tables to process, comma-separated. If not specified then the utility will process all the tables.") diff --git a/utils/zookeeper-create-entry-to-download-part/main.cpp b/utils/zookeeper-create-entry-to-download-part/main.cpp index 1e86fe248ab..b92857929b7 100644 --- a/utils/zookeeper-create-entry-to-download-part/main.cpp +++ b/utils/zookeeper-create-entry-to-download-part/main.cpp @@ -11,7 +11,7 @@ try desc.add_options() ("help,h", "produce help message") ("address,a", boost::program_options::value()->required(), - "addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181") + "addresses of ZooKeeper instances, comma separated. Example: example01e.clickhouse.com:2181") ("path,p", boost::program_options::value()->required(), "path of replica queue to insert node (without trailing slash)") ("name,n", boost::program_options::value()->required(), "name of part to download") ; diff --git a/utils/zookeeper-dump-tree/main.cpp b/utils/zookeeper-dump-tree/main.cpp index 893056564bb..d85762df640 100644 --- a/utils/zookeeper-dump-tree/main.cpp +++ b/utils/zookeeper-dump-tree/main.cpp @@ -14,7 +14,7 @@ int main(int argc, char ** argv) desc.add_options() ("help,h", "produce help message") ("address,a", boost::program_options::value()->required(), - "addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181") + "addresses of ZooKeeper instances, comma separated. Example: example01e.clickhouse.com:2181") ("path,p", boost::program_options::value()->default_value("/"), "where to start") ("ctime,c", "print node ctime") diff --git a/utils/zookeeper-remove-by-list/main.cpp b/utils/zookeeper-remove-by-list/main.cpp index 2c97ffb4a70..cf194bd4861 100644 --- a/utils/zookeeper-remove-by-list/main.cpp +++ b/utils/zookeeper-remove-by-list/main.cpp @@ -12,7 +12,7 @@ try desc.add_options() ("help,h", "produce help message") ("address,a", boost::program_options::value()->required(), - "addresses of ZooKeeper instances, comma separated. Example: example01e.yandex.ru:2181") + "addresses of ZooKeeper instances, comma separated. Example: example01e.clickhouse.com:2181") ; boost::program_options::variables_map options; From ccdd0a60f74e5069efd2c73c332ae281f6c374aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 00:28:56 +0200 Subject: [PATCH 078/110] Remove Arcadia --- base/base/phdr_cache.cpp | 2 +- base/base/phdr_cache.h | 2 +- base/daemon/BaseDaemon.h | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/base/base/phdr_cache.cpp b/base/base/phdr_cache.cpp index 20a755ed7a4..36a9b4f1f62 100644 --- a/base/base/phdr_cache.cpp +++ b/base/base/phdr_cache.cpp @@ -2,7 +2,7 @@ #pragma clang diagnostic ignored "-Wreserved-identifier" #endif -/// This code was based on the code by Fedor Korotkiy (prime@yandex-team.ru) for YT product in Yandex. +/// This code was based on the code by Fedor Korotkiy https://www.linkedin.com/in/fedor-korotkiy-659a1838/ #include diff --git a/base/base/phdr_cache.h b/base/base/phdr_cache.h index d2854ece0bc..b522710c4c4 100644 --- a/base/base/phdr_cache.h +++ b/base/base/phdr_cache.h @@ -1,6 +1,6 @@ #pragma once -/// This code was based on the code by Fedor Korotkiy (prime@yandex-team.ru) for YT product in Yandex. +/// This code was based on the code by Fedor Korotkiy https://www.linkedin.com/in/fedor-korotkiy-659a1838/ /** Collects all dl_phdr_info items and caches them in a static array. * Also rewrites dl_iterate_phdr with a lock-free version which consults the above cache diff --git a/base/daemon/BaseDaemon.h b/base/daemon/BaseDaemon.h index 54a74369dce..152a431922c 100644 --- a/base/daemon/BaseDaemon.h +++ b/base/daemon/BaseDaemon.h @@ -76,10 +76,10 @@ public: /// return none if daemon doesn't exist, reference to the daemon otherwise static std::optional> tryGetInstance() { return tryGetInstance(); } - /// В Graphite компоненты пути(папки) разделяются точкой. - /// У нас принят путь формата root_path.hostname_yandex_ru.key - /// root_path по умолчанию one_min - /// key - лучше группировать по смыслу. Например "meminfo.cached" или "meminfo.free", "meminfo.total" + /// Graphite metric name has components separated by dots. + /// We used to have the following format: root_path.hostname_clickhouse_com.key + /// root_path - one_min by default + /// key - something that makes sense. Examples: "meminfo.cached" or "meminfo.free", "meminfo.total". template void writeToGraphite(const std::string & key, const T & value, const std::string & config_name = DEFAULT_GRAPHITE_CONFIG_NAME, time_t timestamp = 0, const std::string & custom_root_path = "") { From 40357637ec356ec7e65145424a48743b19c1a4ca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 00:33:32 +0200 Subject: [PATCH 079/110] Remove Arcadia --- src/Common/tests/gtest_DateLUTImpl.cpp | 5 +- .../gtest_DateTime64_parsing_and_writing.cpp | 122 ------------------ utils/check-style/check-style | 5 - 3 files changed, 2 insertions(+), 130 deletions(-) diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 1eec83a6ec9..be96c190be3 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -90,10 +90,10 @@ TEST(DateLUTTest, makeDayNumTest) TEST(DateLUTTest, TimeValuesInMiddleOfRange) { - const DateLUTImpl & lut = DateLUT::instance("Europe/Minsk"); + const DateLUTImpl & lut = DateLUT::instance("Asia/Istanbul"); const time_t time = 1568650811; // 2019-09-16 19:20:11 (Monday) - EXPECT_EQ(lut.getTimeZone(), "Europe/Minsk"); + EXPECT_EQ(lut.getTimeZone(), "Asia/Istanbul"); EXPECT_EQ(lut.getOffsetAtStartOfEpoch(), 3600*3); // UTC-3 EXPECT_EQ(lut.toDate(time), 1568581200); @@ -538,4 +538,3 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year1970, // {0, 0 + 11 * 3600 * 24 + 12, 11}, })) ); - diff --git a/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp b/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp index c6208af2d5e..b2fd78fa8c2 100644 --- a/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp +++ b/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp @@ -76,125 +76,3 @@ TEST_P(DateTime64StringParseBestEffortTest, parse) EXPECT_EQ(param.dt64, actual); } - - -// YYYY-MM-DD HH:MM:SS.NNNNNNNNN -INSTANTIATE_TEST_SUITE_P(Basic, - DateTime64StringParseTest, - ::testing::ValuesIn(std::initializer_list{ - { - "When subsecond part is missing from string it is set to zero.", - "2019-09-16 19:20:17", - 1568650817'000, - 3, - DateLUT::instance("Europe/Minsk") - }, - { - "When subsecond part is present in string, but it is zero, it is set to zero.", - "2019-09-16 19:20:17.0", - 1568650817'000, - 3, - DateLUT::instance("Europe/Minsk") - }, - { - "When scale is 0, subsecond part is not set.", - "2019-09-16 19:20:17", - 1568650817ULL, - 0, - DateLUT::instance("Europe/Minsk") - }, - { - "When scale is 0, subsecond part is 0 despite being present in string.", - "2019-09-16 19:20:17.123", - 1568650817ULL, - 0, - DateLUT::instance("Europe/Minsk") - }, - { - "When subsecond part is present in string, it is set correctly to DateTime64 value of scale 3.", - "2019-09-16 19:20:17.123", - 1568650817'123, - 3, - DateLUT::instance("Europe/Minsk") - }, - { - "When subsecond part is present in string (and begins with 0), it is set correctly to DateTime64 value of scale 3.", - "2019-09-16 19:20:17.012", - 1568650817'012, - 3, - DateLUT::instance("Europe/Minsk") - }, - { - "When subsecond part scale is smaller than DateTime64 scale, subsecond part is properly adjusted (as if padded from right with zeroes).", - "2019-09-16 19:20:17.123", - 1568650817'12300ULL, - 5, - DateLUT::instance("Europe/Minsk") - }, - { - "When subsecond part scale is larger than DateTime64 scale, subsecond part is truncated.", - "2019-09-16 19:20:17.123", - 1568650817'1ULL, - 1, - DateLUT::instance("Europe/Minsk") - } - }) -); - -INSTANTIATE_TEST_SUITE_P(BestEffort, - DateTime64StringParseBestEffortTest, - ::testing::ValuesIn(std::initializer_list{ - { - "When subsecond part is unreasonably large, it truncated to given scale", - "2019-09-16 19:20:17.12345678910111213141516171819202122233435363738393031323334353637383940414243444546474849505152535455565758596061626364", - 1568650817'123456ULL, - 6, - DateLUT::instance("Europe/Minsk") - } - }) -); - - -// TODO: add negative test cases for invalid strings, verifying that error is reported properly - -INSTANTIATE_TEST_SUITE_P(Basic, - DateTime64StringWriteTest, - ::testing::ValuesIn(std::initializer_list{ - { - "non-zero subsecond part on DateTime64 with scale of 3", - "2019-09-16 19:20:17.123", - 1568650817'123, - 3, - DateLUT::instance("Europe/Minsk") - }, - { - "non-zero subsecond part on DateTime64 with scale of 5", - "2019-09-16 19:20:17.12345", - 1568650817'12345ULL, - 5, - DateLUT::instance("Europe/Minsk") - }, - { - "Zero subsecond part is written to string", - "2019-09-16 19:20:17.000", - 1568650817'000ULL, - 3, - DateLUT::instance("Europe/Minsk") - }, - { - "When scale is 0, subsecond part (and separtor) is missing from string", - "2019-09-16 19:20:17", - 1568650817ULL, - 0, - DateLUT::instance("Europe/Minsk") - }, - { - "Subsecond part with leading zeroes is written to string correctly", - "2019-09-16 19:20:17.001", - 1568650817'001ULL, - 3, - DateLUT::instance("Europe/Minsk") - } - }) -); - diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 6ebf53cb932..20954781fbd 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -147,11 +147,6 @@ find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*. grep -vP $EXCLUDE_DIRS | xargs yamllint --config-file=$ROOT_PATH/.yamllint -# Machine translation to Russian is strictly prohibited -find $ROOT_PATH/docs/ru -name '*.md' | - grep -vP $EXCLUDE_DIRS | - xargs grep -l -F 'machine_translated: true' - # Tests should not be named with "fail" in their names. It makes looking at the results less convenient. find $ROOT_PATH/tests/queries -iname '*fail*' | grep -vP $EXCLUDE_DIRS | From 69b91c5410dc5d1153be96b2985a5b78462028e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 02:08:06 +0200 Subject: [PATCH 080/110] Simplification #36313 --- src/Common/getNumberOfPhysicalCPUCores.cpp | 34 +++++++--------------- 1 file changed, 10 insertions(+), 24 deletions(-) diff --git a/src/Common/getNumberOfPhysicalCPUCores.cpp b/src/Common/getNumberOfPhysicalCPUCores.cpp index f415ee58e85..0df7a83cd53 100644 --- a/src/Common/getNumberOfPhysicalCPUCores.cpp +++ b/src/Common/getNumberOfPhysicalCPUCores.cpp @@ -5,9 +5,6 @@ # include # include #endif -#if USE_CPUID -# include -#endif #include @@ -41,32 +38,21 @@ static unsigned getCGroupLimitedCPUCores(unsigned default_cpu_count) static unsigned getNumberOfPhysicalCPUCoresImpl() { - unsigned cpu_count = 0; // start with an invalid num + unsigned cpu_count = std::thread::hardware_concurrency(); -#if USE_CPUID - cpu_raw_data_t raw_data; - cpu_id_t data; + /// Most of x86_64 CPUs have 2-way Hyper-Threading + /// Aarch64 and RISC-V don't have SMT so far. + /// POWER has SMT and it can be multiple way (like 8-way), but we don't know how ClickHouse really behaves, so use all of them. - /// On Xen VMs, libcpuid returns wrong info (zero number of cores). Fallback to alternative method. - /// Also, libcpuid does not support some CPUs like AMD Hygon C86 7151. - /// Also, libcpuid gives strange result on Google Compute Engine VMs. - /// Example: - /// num_cores = 12, /// number of physical cores on current CPU socket - /// total_logical_cpus = 1, /// total number of logical cores on all sockets - /// num_logical_cpus = 24. /// number of logical cores on current CPU socket - /// It means two-way hyper-threading (24 / 12), but contradictory, 'total_logical_cpus' == 1. - - if (0 == cpuid_get_raw_data(&raw_data) && 0 == cpu_identify(&raw_data, &data) && data.num_logical_cpus != 0) - cpu_count = data.num_cores * data.total_logical_cpus / data.num_logical_cpus; +#if defined(__x86_64__) + /// Let's limit ourself to the number of physical cores. + /// But if the number of logical cores is small - maybe it is a small machine + /// or very limited cloud instance and it is reasonable to use all the cores. + if (cpu_count >= 8) + cpu_count /= 2; #endif - /// As a fallback (also for non-x86 architectures) assume there are no hyper-threading on the system. - /// (Actually, only Aarch64 is supported). - if (cpu_count == 0) - cpu_count = std::thread::hardware_concurrency(); - #if defined(OS_LINUX) - /// TODO: add a setting for disabling that, similar to UseContainerSupport in java cpu_count = getCGroupLimitedCPUCores(cpu_count); #endif From 33ffdcaa4f21f6611eeea8b72c0cf71971baf86d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 02:18:22 +0200 Subject: [PATCH 081/110] Return two changes --- src/Common/tests/gtest_DateLUTImpl.cpp | 5 +- .../gtest_DateTime64_parsing_and_writing.cpp | 122 ++++++++++++++++++ 2 files changed, 125 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index be96c190be3..1eec83a6ec9 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -90,10 +90,10 @@ TEST(DateLUTTest, makeDayNumTest) TEST(DateLUTTest, TimeValuesInMiddleOfRange) { - const DateLUTImpl & lut = DateLUT::instance("Asia/Istanbul"); + const DateLUTImpl & lut = DateLUT::instance("Europe/Minsk"); const time_t time = 1568650811; // 2019-09-16 19:20:11 (Monday) - EXPECT_EQ(lut.getTimeZone(), "Asia/Istanbul"); + EXPECT_EQ(lut.getTimeZone(), "Europe/Minsk"); EXPECT_EQ(lut.getOffsetAtStartOfEpoch(), 3600*3); // UTC-3 EXPECT_EQ(lut.toDate(time), 1568581200); @@ -538,3 +538,4 @@ INSTANTIATE_TEST_SUITE_P(AllTimezones_Year1970, // {0, 0 + 11 * 3600 * 24 + 12, 11}, })) ); + diff --git a/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp b/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp index b2fd78fa8c2..c6208af2d5e 100644 --- a/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp +++ b/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp @@ -76,3 +76,125 @@ TEST_P(DateTime64StringParseBestEffortTest, parse) EXPECT_EQ(param.dt64, actual); } + + +// YYYY-MM-DD HH:MM:SS.NNNNNNNNN +INSTANTIATE_TEST_SUITE_P(Basic, + DateTime64StringParseTest, + ::testing::ValuesIn(std::initializer_list{ + { + "When subsecond part is missing from string it is set to zero.", + "2019-09-16 19:20:17", + 1568650817'000, + 3, + DateLUT::instance("Europe/Minsk") + }, + { + "When subsecond part is present in string, but it is zero, it is set to zero.", + "2019-09-16 19:20:17.0", + 1568650817'000, + 3, + DateLUT::instance("Europe/Minsk") + }, + { + "When scale is 0, subsecond part is not set.", + "2019-09-16 19:20:17", + 1568650817ULL, + 0, + DateLUT::instance("Europe/Minsk") + }, + { + "When scale is 0, subsecond part is 0 despite being present in string.", + "2019-09-16 19:20:17.123", + 1568650817ULL, + 0, + DateLUT::instance("Europe/Minsk") + }, + { + "When subsecond part is present in string, it is set correctly to DateTime64 value of scale 3.", + "2019-09-16 19:20:17.123", + 1568650817'123, + 3, + DateLUT::instance("Europe/Minsk") + }, + { + "When subsecond part is present in string (and begins with 0), it is set correctly to DateTime64 value of scale 3.", + "2019-09-16 19:20:17.012", + 1568650817'012, + 3, + DateLUT::instance("Europe/Minsk") + }, + { + "When subsecond part scale is smaller than DateTime64 scale, subsecond part is properly adjusted (as if padded from right with zeroes).", + "2019-09-16 19:20:17.123", + 1568650817'12300ULL, + 5, + DateLUT::instance("Europe/Minsk") + }, + { + "When subsecond part scale is larger than DateTime64 scale, subsecond part is truncated.", + "2019-09-16 19:20:17.123", + 1568650817'1ULL, + 1, + DateLUT::instance("Europe/Minsk") + } + }) +); + +INSTANTIATE_TEST_SUITE_P(BestEffort, + DateTime64StringParseBestEffortTest, + ::testing::ValuesIn(std::initializer_list{ + { + "When subsecond part is unreasonably large, it truncated to given scale", + "2019-09-16 19:20:17.12345678910111213141516171819202122233435363738393031323334353637383940414243444546474849505152535455565758596061626364", + 1568650817'123456ULL, + 6, + DateLUT::instance("Europe/Minsk") + } + }) +); + + +// TODO: add negative test cases for invalid strings, verifying that error is reported properly + +INSTANTIATE_TEST_SUITE_P(Basic, + DateTime64StringWriteTest, + ::testing::ValuesIn(std::initializer_list{ + { + "non-zero subsecond part on DateTime64 with scale of 3", + "2019-09-16 19:20:17.123", + 1568650817'123, + 3, + DateLUT::instance("Europe/Minsk") + }, + { + "non-zero subsecond part on DateTime64 with scale of 5", + "2019-09-16 19:20:17.12345", + 1568650817'12345ULL, + 5, + DateLUT::instance("Europe/Minsk") + }, + { + "Zero subsecond part is written to string", + "2019-09-16 19:20:17.000", + 1568650817'000ULL, + 3, + DateLUT::instance("Europe/Minsk") + }, + { + "When scale is 0, subsecond part (and separtor) is missing from string", + "2019-09-16 19:20:17", + 1568650817ULL, + 0, + DateLUT::instance("Europe/Minsk") + }, + { + "Subsecond part with leading zeroes is written to string correctly", + "2019-09-16 19:20:17.001", + 1568650817'001ULL, + 3, + DateLUT::instance("Europe/Minsk") + } + }) +); + From fa46e0d22fb28f62a7125f680c35b7a25edb8b9f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 02:54:21 +0200 Subject: [PATCH 082/110] Add an option for build profiling --- CMakeLists.txt | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index dad9a25ab26..4f1a6c05730 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -314,6 +314,15 @@ if (ENABLE_BUILD_PATH_MAPPING) set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -ffile-prefix-map=${CMAKE_SOURCE_DIR}=.") endif () +option (ENABLE_BUILD_PROFILING "Enable profiling of build time" OFF) +if (ENABLE_BUILD_PROFILING) + if (COMPILER_CLANG) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -ftime-trace") + else () + message (${RECONFIGURE_MESSAGE_LEVEL} "Build profiling is only available with CLang") + endif () +endif () + if (${CMAKE_VERSION} VERSION_LESS "3.12.4") # CMake < 3.12 doesn't support setting 20 as a C++ standard version. # We will add C++ standard controlling flag in CMAKE_CXX_FLAGS manually for now. From 294efeccfe7532fe1c29052b7a908e245a0a76ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Apr 2022 01:15:40 +0200 Subject: [PATCH 083/110] Fix clang-tidy-14 (part 1) --- contrib/sysroot | 2 +- src/Access/DiskAccessStorage.cpp | 3 ++- .../AggregateFunctionAvgWeighted.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionDeltaSum.cpp | 2 +- .../AggregateFunctionGroupArray.cpp | 2 +- .../AggregateFunctionGroupBitmap.cpp | 2 +- .../AggregateFunctionGroupUniqArray.cpp | 2 +- src/AggregateFunctions/AggregateFunctionQuantile.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionSparkbar.cpp | 2 +- .../AggregateFunctionStatisticsSimple.cpp | 2 +- src/AggregateFunctions/AggregateFunctionSum.cpp | 2 +- src/AggregateFunctions/AggregateFunctionSumCount.cpp | 2 +- src/AggregateFunctions/AggregateFunctionTopK.cpp | 2 +- src/Common/UTF8Helpers.cpp | 2 +- src/Dictionaries/IPAddressDictionary.cpp | 12 ++++++------ src/Functions/FunctionHelpers.cpp | 2 +- src/Functions/FunctionsLogical.cpp | 6 +++--- src/Functions/array/arrayElement.cpp | 2 +- src/Functions/if.cpp | 6 +++--- src/Functions/toLowCardinality.cpp | 1 - src/Interpreters/JIT/compileFunction.cpp | 2 +- src/Interpreters/convertFieldToType.cpp | 12 ++++++------ src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp | 3 ++- src/Storages/FileLog/StorageFileLog.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp | 2 +- .../MergeTree/MergeTreeIndexGranuleBloomFilter.cpp | 2 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- 29 files changed, 46 insertions(+), 45 deletions(-) diff --git a/contrib/sysroot b/contrib/sysroot index bbcac834526..e9fb375d0a1 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit bbcac834526d90d1e764164b861be426891d1743 +Subproject commit e9fb375d0a1e5ebfd74c043f088f2342552103f8 diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 7393fcd8d36..a9eb27c291c 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -327,7 +327,8 @@ void DiskAccessStorage::scheduleWriteLists(AccessEntityType type) /// Create the 'need_rebuild_lists.mark' file. /// This file will be used later to find out if writing lists is successful or not. - std::ofstream{getNeedRebuildListsMarkFilePath(directory_path)}; + std::ofstream out{getNeedRebuildListsMarkFilePath(directory_path)}; + out.close(); lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this}; lists_writing_thread_is_waiting = true; diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index ab6fdc8fd7e..4d7901a7fac 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -39,7 +39,7 @@ bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept } template -static IAggregateFunction * create(const IDataType & second_type, TArgs && ... args) +IAggregateFunction * create(const IDataType & second_type, TArgs && ... args) { const WhichDataType which(second_type); @@ -51,7 +51,7 @@ static IAggregateFunction * create(const IDataType & second_type, TArgs && ... a // Not using helper functions because there are no templates for binary decimal/numeric function. template -static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args) +IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args) { const WhichDataType which(first_type); diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSum.cpp b/src/AggregateFunctions/AggregateFunctionDeltaSum.cpp index f1c6e7c6112..3b43d9a85f8 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSum.cpp +++ b/src/AggregateFunctions/AggregateFunctionDeltaSum.cpp @@ -30,7 +30,7 @@ AggregateFunctionPtr createAggregateFunctionDeltaSum( throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - DataTypePtr data_type = arguments[0]; + const DataTypePtr & data_type = arguments[0]; if (isInteger(data_type) || isFloat(data_type)) return AggregateFunctionPtr(createWithNumericType( diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 5a9fd778277..85075d5a4d6 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -20,7 +20,7 @@ namespace { template