From 18e8f0eb5e51f2851e4e0a04282cb1d60a2a21c5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 4 Jun 2021 09:43:56 +0300 Subject: [PATCH 01/43] Add ability to push down LIMIT for distributed queries This way the remote nodes will not need to send all the rows, so this will decrease network io and also this will make queries w/ optimize_aggregation_in_order=1/LIMIT X and w/o ORDER BY faster since it initiator will not need to read all the rows, only first X (but note that for this you need to your data to be sharded correctly or you may get inaccurate results). Note, that having lots of processing stages will increase the complexity of interpreter (it is already not that clean and simple right now). Although using separate QueryProcessingStage looks pretty natural. Another option is to make WithMergeableStateAfterAggregation always, but in this case you will not be able to disable only this optimization, i.e. if there will be some issue with it. v2: fix OFFSET v3: convert 01814_distributed_push_down_limit test to .sh and add retries v4: add test with OFFSET v5: add new query stage into the bash completion v6/tests: use LIMIT O,L syntax over LIMIT L OFFSET O since it is broken in ANTLR parser https://clickhouse-test-reports.s3.yandex.net/23027/a18a06399b7aeacba7c50b5d1e981ada5df19745/functional_stateless_tests_(antlr_debug).html#fail1 v7/tests: set use_hedged_requests to 0, to avoid excessive log entries on retries https://clickhouse-test-reports.s3.yandex.net/23027/a18a06399b7aeacba7c50b5d1e981ada5df19745/functional_stateless_tests_flaky_check_(address).html#fail1 --- docs/en/operations/settings/settings.md | 12 ++ .../completions/clickhouse-bootstrap | 1 + programs/benchmark/Benchmark.cpp | 2 +- programs/client/Client.cpp | 2 +- src/Core/QueryProcessingStage.cpp | 2 + src/Core/QueryProcessingStage.h | 10 +- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 22 +-- .../getHeaderForProcessingStage.cpp | 1 + src/Storages/StorageDistributed.cpp | 19 +- ...1814_distributed_push_down_limit.reference | 37 ++++ .../01814_distributed_push_down_limit.sh | 167 ++++++++++++++++++ ...tate_after_aggregation_and_limit.reference | 1 + ...eable_state_after_aggregation_and_limit.sh | 8 + .../queries/0_stateless/arcadia_skip_list.txt | 1 + 15 files changed, 270 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/01814_distributed_push_down_limit.reference create mode 100755 tests/queries/0_stateless/01814_distributed_push_down_limit.sh create mode 100644 tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.reference create mode 100755 tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.sh diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 10461eacbff..59417753db0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1578,6 +1578,18 @@ FORMAT PrettyCompactMonoBlock Default value: 0 +## distributed_push_down_limit (#distributed-push-down-limit} + +LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT. + +Possible values: + +- 0 - Disabled +- 1 - Enabled + +!!! note "Note" + That with this setting the result of the query may be inaccurate. + ## optimize_skip_unused_shards_limit {#optimize-skip-unused-shards-limit} Limit for number of sharding key values, turns off `optimize_skip_unused_shards` if the limit is reached. diff --git a/programs/bash-completion/completions/clickhouse-bootstrap b/programs/bash-completion/completions/clickhouse-bootstrap index 7109148a192..793d47501dd 100644 --- a/programs/bash-completion/completions/clickhouse-bootstrap +++ b/programs/bash-completion/completions/clickhouse-bootstrap @@ -20,6 +20,7 @@ CLICKHOUSE_QueryProcessingStage=( fetch_columns with_mergeable_state with_mergeable_state_after_aggregation + with_mergeable_state_after_aggregation_and_limit ) CLICKHOUSE_Format=( diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 2e48c5d20c5..c8f1a4eef47 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -580,7 +580,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("query", value()->default_value(""), "query to execute") ("concurrency,c", value()->default_value(1), "number of parallel queries") ("delay,d", value()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)") - ("stage", value()->default_value("complete"), "request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation") + ("stage", value()->default_value("complete"), "request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("iterations,i", value()->default_value(0), "amount of queries to be executed") ("timelimit,t", value()->default_value(0.), "stop launch of queries after specified time limit") ("randomize,r", value()->default_value(false), "randomize order of execution") diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f268d2b5cdc..2000f04877f 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -2463,7 +2463,7 @@ public: ("password", po::value()->implicit_value("\n", ""), "password") ("ask-password", "ask-password") ("quota_key", po::value(), "A string to differentiate quotas when the user have keyed quotas configured on server") - ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation") + ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("query_id", po::value(), "query_id") ("query,q", po::value(), "query") ("database,d", po::value(), "database") diff --git a/src/Core/QueryProcessingStage.cpp b/src/Core/QueryProcessingStage.cpp index 14bde0e548d..b5b837e1f61 100644 --- a/src/Core/QueryProcessingStage.cpp +++ b/src/Core/QueryProcessingStage.cpp @@ -24,6 +24,8 @@ namespace QueryProcessingStage stage = WithMergeableState; else if (stage_string == "with_mergeable_state_after_aggregation") stage = WithMergeableStateAfterAggregation; + else if (stage_string == "with_mergeable_state_after_aggregation_and_limit") + stage = WithMergeableStateAfterAggregationAndLimit; else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query processing stage: {}", stage_string); diff --git a/src/Core/QueryProcessingStage.h b/src/Core/QueryProcessingStage.h index 01e7e12ab1e..7ccaa17eaed 100644 --- a/src/Core/QueryProcessingStage.h +++ b/src/Core/QueryProcessingStage.h @@ -26,8 +26,15 @@ namespace QueryProcessingStage /// It is used for auto distributed_group_by_no_merge optimization for distributed engine. /// (See comments in StorageDistributed). WithMergeableStateAfterAggregation = 3, + /// Same as WithMergeableStateAfterAggregation but also will apply limit on each shard. + /// + /// This query stage will be used for auto + /// distributed_group_by_no_merge/distributed_push_down_limit + /// optimization. + /// (See comments in StorageDistributed). + WithMergeableStateAfterAggregationAndLimit = 4, - MAX = 4, + MAX = 5, }; inline const char * toString(UInt64 stage) @@ -38,6 +45,7 @@ namespace QueryProcessingStage "WithMergeableState", "Complete", "WithMergeableStateAfterAggregation", + "WithMergeableStateAfterAggregationAndLimit", }; return stage < MAX ? data[stage] diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 08b69e52bbc..fb3ba58733b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -118,6 +118,7 @@ class IColumn; \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \ + M(UInt64, distributed_push_down_limit, 0, "If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0f8f381cea7..28d4c425f1a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -632,7 +632,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() /// Running on the initiating server during distributed processing or if query is not distributed. /// /// Also note that with distributed_group_by_no_merge=2 (i.e. when optimize_distributed_group_by_sharding_key takes place) - /// the query on the remote server will be processed up to WithMergeableStateAfterAggregation, + /// the query on the remote server will be processed up to WithMergeableStateAfterAggregationAndLimit, /// So it will do partial second stage (second_stage=true), and initiator will do the final part. bool second_stage = from_stage <= QueryProcessingStage::WithMergeableState && options.to_stage > QueryProcessingStage::WithMergeableState; @@ -704,7 +704,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() return res; } - if (options.to_stage == QueryProcessingStage::Enum::WithMergeableStateAfterAggregation) + if (options.to_stage >= QueryProcessingStage::Enum::WithMergeableStateAfterAggregation) { // It's different from selected_columns, see the comment above for // WithMergeableState stage. @@ -1011,10 +1011,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu /// Support optimize_distributed_group_by_sharding_key /// Is running on the initiating server during distributed processing? - if (from_stage == QueryProcessingStage::WithMergeableStateAfterAggregation) + if (from_stage >= QueryProcessingStage::WithMergeableStateAfterAggregation) from_aggregation_stage = true; /// Is running on remote servers during distributed processing? - if (options.to_stage == QueryProcessingStage::WithMergeableStateAfterAggregation) + if (options.to_stage >= QueryProcessingStage::WithMergeableStateAfterAggregation) to_aggregation_stage = true; /// Read the data from Storage. from_stage - to what stage the request was completed in Storage. @@ -1300,7 +1300,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu * but there is no aggregation, then on the remote servers ORDER BY was made * - therefore, we merge the sorted streams from remote servers. * - * Also in case of remote servers was process the query up to WithMergeableStateAfterAggregation + * Also in case of remote servers was process the query up to WithMergeableStateAfterAggregationAndLimit * (distributed_group_by_no_merge=2 or optimize_distributed_group_by_sharding_key=1 takes place), * then merge the sorted streams is enough, since remote servers already did full ORDER BY. */ @@ -1334,13 +1334,15 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } } + bool apply_limit = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregation; + bool apply_offset = options.to_stage != QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; bool has_prelimit = false; - if (!to_aggregation_stage && + if (apply_limit && query.limitLength() && !query.limit_with_ties && !hasWithTotalsInAnySubqueryInFromClause(query) && !query.arrayJoinExpressionList() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes && !has_withfill) { - executePreLimit(query_plan, false); + executePreLimit(query_plan, /* do_not_skip_offset= */!apply_offset); has_prelimit = true; } @@ -1367,7 +1369,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } /// Projection not be done on the shards, since then initiator will not find column in blocks. - /// (significant only for WithMergeableStateAfterAggregation). + /// (significant only for WithMergeableStateAfterAggregation/WithMergeableStateAfterAggregationAndLimit). if (!to_aggregation_stage) { /// We must do projection after DISTINCT because projection may remove some columns. @@ -1378,10 +1380,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu executeExtremes(query_plan); /// Limit is no longer needed if there is prelimit. - if (!to_aggregation_stage && !has_prelimit) + if (apply_limit && !has_prelimit) executeLimit(query_plan); - if (!to_aggregation_stage) + if (apply_offset) executeOffset(query_plan); } } diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 9c7c86a0b88..e77b64dea48 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -69,6 +69,7 @@ Block getHeaderForProcessingStage( case QueryProcessingStage::WithMergeableState: case QueryProcessingStage::Complete: case QueryProcessingStage::WithMergeableStateAfterAggregation: + case QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit: case QueryProcessingStage::MAX: { auto query = query_info.query->clone(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ace0963bc0a..a1aacf95105 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -288,6 +288,7 @@ void replaceConstantExpressions( /// is one of the following: /// - QueryProcessingStage::Complete /// - QueryProcessingStage::WithMergeableStateAfterAggregation +/// - QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit /// - none (in this case regular WithMergeableState should be used) std::optional getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, bool extremes, const Block & sharding_key_block) { @@ -349,13 +350,13 @@ std::optional getOptimizedQueryProcessingStage(const // ORDER BY const ASTPtr order_by = select.orderBy(); if (order_by) - return QueryProcessingStage::WithMergeableStateAfterAggregation; + return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; // LIMIT BY // LIMIT // OFFSET if (select.limitBy() || select.limitLength() || select.limitOffset()) - return QueryProcessingStage::WithMergeableStateAfterAggregation; + return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; // Only simple SELECT FROM GROUP BY sharding_key can use Complete state. return QueryProcessingStage::Complete; @@ -514,11 +515,23 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage( if (settings.distributed_group_by_no_merge) { if (settings.distributed_group_by_no_merge == DISTRIBUTED_GROUP_BY_NO_MERGE_AFTER_AGGREGATION) - return QueryProcessingStage::WithMergeableStateAfterAggregation; + { + if (settings.distributed_push_down_limit) + return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; + else + return QueryProcessingStage::WithMergeableStateAfterAggregation; + } else + { + /// NOTE: distributed_group_by_no_merge=1 does not respect distributed_push_down_limit + /// (since in this case queries processed separatelly and the initiator is just a proxy in this case). return QueryProcessingStage::Complete; + } } + if (settings.distributed_push_down_limit) + return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit; + /// Nested distributed query cannot return Complete stage, /// since the parent query need to aggregate the results after. if (to_stage == QueryProcessingStage::WithMergeableState) diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.reference b/tests/queries/0_stateless/01814_distributed_push_down_limit.reference new file mode 100644 index 00000000000..f879f2cbd21 --- /dev/null +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.reference @@ -0,0 +1,37 @@ +distributed_push_down_limit=0 +100 100 +distributed_push_down_limit=1 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +40 40 +auto-distributed_push_down_limit +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +40 40 +distributed_push_down_limit=1 with OFFSET +97 +96 +96 +95 +95 +94 +94 +93 +93 +92 diff --git a/tests/queries/0_stateless/01814_distributed_push_down_limit.sh b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh new file mode 100755 index 00000000000..93321646037 --- /dev/null +++ b/tests/queries/0_stateless/01814_distributed_push_down_limit.sh @@ -0,0 +1,167 @@ +#!/usr/bin/env bash +# shellcheck disable=SC2206 + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# -- NOTE: this test cannot use 'current_database = $CLICKHOUSE_DATABASE', +# -- because it does not propagated via remote queries, +# -- hence it uses query_id/initial_query_id. + +function setup() +{ + $CLICKHOUSE_CLIENT -nm -q " + drop table if exists data_01814; + drop table if exists dist_01814; + + create table data_01814 (key Int) Engine=MergeTree() order by key settings index_granularity=10 as select * from numbers(100); + create table dist_01814 as data_01814 engine=Distributed('test_cluster_two_shards', $CLICKHOUSE_DATABASE, data_01814, key); + " +} + +function cleanup() +{ + $CLICKHOUSE_CLIENT -nm -q " + drop table data_01814; + drop table dist_01814; + " +} + +function make_query_id() +{ + echo "$(tr -cd '[:lower:]' < /dev/urandom | head -c10)-$CLICKHOUSE_DATABASE" +} + +function test_distributed_push_down_limit_with_query_log() +{ + local table=$1 && shift + local offset=$1 && shift + local query_id + + query_id="$(make_query_id)" + + # NOTES: + # - max_rows_to_read_leaf cannot be used since it does not know anything + # about optimize_aggregation_in_order, + # - limit push down can be checked only with optimize_aggregation_in_order, + # since otherwise the query will be canceled too early, and read_rows will be + # small. + local settings_and_opts=( + --query_id "$query_id" + + --max_block_size 20 + --optimize_aggregation_in_order 1 + --log_queries 1 + --log_queries_min_type 'QUERY_FINISH' + + # disable hedged requests to avoid excessive log entries + --use_hedged_requests 0 + + "$@" + ) + + $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from $table group by key limit $offset, 10" + + $CLICKHOUSE_CLIENT -nm -q " + system flush logs; + select read_rows from system.query_log + where + event_date = today() + and query_kind = 'Select' /* exclude DESC TABLE */ + and initial_query_id = '$query_id' and initial_query_id != query_id; + " | xargs # convert new lines to spaces +} + +function test_distributed_push_down_limit_0() +{ + local args=( + "remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814)" + 0 # offset + --distributed_push_down_limit 0 + ) + test_distributed_push_down_limit_with_query_log "${args[@]}" "$@" +} + +function test_distributed_push_down_limit_1() +{ + local args=( + "remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814)" + 0 # offset + --distributed_push_down_limit 1 + ) + test_distributed_push_down_limit_with_query_log "${args[@]}" +} + +function test_distributed_push_down_limit_1_offset() +{ + local settings_and_opts=( + --distributed_push_down_limit 1 + ) + + $CLICKHOUSE_CLIENT "${settings_and_opts[@]}" -q "select * from remote('127.{2,3}', $CLICKHOUSE_DATABASE, data_01814) group by key order by key desc limit 5, 10" +} + +function test_auto_distributed_push_down_limit() +{ + local args=( + dist_01814 + 0 # offset + --optimize_skip_unused_shards 1 + --optimize_distributed_group_by_sharding_key 1 + --prefer_localhost_replica 0 + --distributed_push_down_limit 0 + ) + test_distributed_push_down_limit_with_query_log "${args[@]}" +} + +function main() +{ + setup + trap cleanup EXIT + + echo 'distributed_push_down_limit=0' + test_distributed_push_down_limit_0 --format Null + + # + # The following tests (tests with distributed_push_down_limit=1) requires + # retries, since the query may be canceled earlier due to LIMIT, and so + # only one shard will be processed, and it will get not 40 but 20 rows: + # + # 1.160920 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} executeQuery: (from [::ffff:127.0.0.1]:42778, initial_query_id: 66cf643c-b1b4-4f7e-942a-c4c3493029f6, using production parser) (comment: /usr/share/clickhouse-test/queries/0_stateless/01814_distributed_push_down_limit.sql) WITH CAST('test_31uut9', 'String') AS id_distributed_push_down_limit_1 SELECT key FROM test_31uut9.data_01814 GROUP BY key LIMIT 10 + # 1.214964 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} ContextAccess (default): Access granted: SELECT(key) ON test_31uut9.data_01814 + # 1.216790 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} test_31uut9.data_01814 (b484ad2e-0591-4faf-8110-1dcbd7cdd0db) (SelectExecutor): Key condition: unknown + # 1.227245 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} test_31uut9.data_01814 (b484ad2e-0591-4faf-8110-1dcbd7cdd0db) (SelectExecutor): Selected 1/1 parts by partition key, 1 parts by primary key, 10/11 marks by primary key, 10 marks to read from 1 ranges + # 1.228452 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} MergeTreeSelectProcessor: Reading 3 ranges from part all_1_1_0, approx. 100 rows starting from 0 + # 1.229104 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} InterpreterSelectQuery: FetchColumns -> WithMergeableStateAfterAggregationAndLimit + # 1.339085 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} TCPHandler: Query was cancelled. + # 1.416573 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} executeQuery: Read 20 rows, 80.00 B in 0.254374666 sec., 78 rows/sec., 314.50 B/sec. + # 1.419006 [ 291 ] {7ac5de70-c26c-4e3b-bdee-3873ad1b84f1} MemoryTracker: Peak memory usage (for query): 0.00 B. + # + + local out out_lines max_tries=20 + + echo 'distributed_push_down_limit=1' + for ((i = 0; i < max_tries; ++i)); do + out=$(test_distributed_push_down_limit_1) + out_lines=( $out ) + if [[ ${#out_lines[@]} -gt 2 ]] && [[ ${out_lines[-1]} = 40 ]] && [[ ${out_lines[-2]} = 40 ]]; then + break + fi + done + echo "$out" + + echo 'auto-distributed_push_down_limit' + for ((i = 0; i < max_tries; ++i)); do + out=$(test_auto_distributed_push_down_limit) + out_lines=( $out ) + if [[ ${#out_lines[@]} -gt 2 ]] && [[ ${out_lines[-1]} = 40 ]] && [[ ${out_lines[-2]} = 40 ]]; then + break + fi + done + echo "$out" + + echo 'distributed_push_down_limit=1 with OFFSET' + test_distributed_push_down_limit_1_offset +} +main "$@" diff --git a/tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.reference b/tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.sh b/tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.sh new file mode 100755 index 00000000000..0efacc4ac31 --- /dev/null +++ b/tests/queries/0_stateless/01815_with_mergeable_state_after_aggregation_and_limit.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# with_mergeable_state_after_aggregation will not stop after 1 row, while with_mergeable_state_after_aggregation_and_limit should +$CLICKHOUSE_CLIENT -q 'select * from system.numbers limit 1' --stage with_mergeable_state_after_aggregation_and_limit diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index dac43ff5d4b..5f6ec299888 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -234,6 +234,7 @@ 01801_distinct_group_by_shard 01804_dictionary_decimal256_type 01801_s3_distributed +01814_distributed_push_down_limit 01833_test_collation_alvarotuso 01850_dist_INSERT_preserve_error 01870_modulo_partition_key From 235e3e2f5ba2e6061fbeac1e282f0fc9f7d3b9b5 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 20 May 2021 16:47:12 +0300 Subject: [PATCH 02/43] Support structs in Arrow/Parquet/ORC --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 48 ++++++++++++- .../Formats/Impl/CHColumnToArrowColumn.cpp | 51 ++++++++++++- .../Formats/Impl/ORCBlockInputFormat.cpp | 2 +- .../Formats/Impl/ORCBlockOutputFormat.cpp | 71 +++++++++++++++---- .../Formats/Impl/ORCBlockOutputFormat.h | 4 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 23 +++++- .../00900_orc_tuples_load.reference | 2 + .../0_stateless/00900_orc_tuples_load.sh | 17 +++++ .../00900_parquet_tuples_load.reference | 2 + .../0_stateless/00900_parquet_tuples_load.sh | 17 +++++ .../0_stateless/01273_arrow_arrays_load.sh | 2 +- .../01273_arrow_tuples_load.reference | 2 + .../0_stateless/01273_arrow_tuples_load.sh | 17 +++++ 13 files changed, 237 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/00900_orc_tuples_load.reference create mode 100755 tests/queries/0_stateless/00900_orc_tuples_load.sh create mode 100644 tests/queries/0_stateless/00900_parquet_tuples_load.reference create mode 100755 tests/queries/0_stateless/00900_parquet_tuples_load.sh create mode 100644 tests/queries/0_stateless/01273_arrow_tuples_load.reference create mode 100755 tests/queries/0_stateless/01273_arrow_tuples_load.sh diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 29fff1fd4e0..23f356a77fc 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -7,15 +7,17 @@ #include #include #include +#include +#include #include #include #include #include #include #include +#include #include #include -#include namespace DB @@ -309,8 +311,6 @@ namespace DB break; case arrow::Type::LIST: { - const auto * list_type = static_cast(arrow_column->type().get()); - auto list_nested_type = list_type->value_type(); arrow::ArrayVector array_vector; array_vector.reserve(arrow_column->num_chunks()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -326,6 +326,25 @@ namespace DB fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn()); break; } + case arrow::Type::STRUCT: + { + ColumnTuple & column_tuple = typeid_cast(internal_column); + int fields_count = column_tuple.tupleSize(); + std::vector nested_arrow_columns(fields_count); + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::StructArray & struct_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + for (int i = 0; i < fields_count; ++i) + nested_arrow_columns[i].emplace_back(struct_chunk.field(i)); + } + + for (int i = 0; i != fields_count; ++i) + { + auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); + readColumnFromArrowColumn(nested_arrow_column, column_tuple.getColumn(i), column_name, format_name, false); + } + break; + } # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ fillColumnWithNumericData(arrow_column, internal_column); \ @@ -372,6 +391,29 @@ namespace DB return std::make_shared(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name)); } + if (arrow_type->id() == arrow::Type::STRUCT) + { + const auto * struct_type = static_cast(arrow_type.get()); + const DataTypeTuple * tuple_type = typeid_cast(column_type.get()); + if (!tuple_type) + throw Exception{"Cannot convert arrow STRUCT type to a not Tuple ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + + const DataTypes & tuple_nested_types = tuple_type->getElements(); + int internal_fields_num = tuple_nested_types.size(); + /// If internal column has less elements then arrow struct, we will select only first internal_fields_num columns. + if (internal_fields_num > struct_type->num_fields()) + throw Exception{ + "Cannot convert arrow STRUCT with " + std::to_string(struct_type->num_fields()) + " fields to a ClickHouse Tuple with " + + std::to_string(internal_fields_num) + " elements " + column_type->getName(), + ErrorCodes::CANNOT_CONVERT_TYPE}; + + DataTypes nested_types; + for (int i = 0; i < internal_fields_num; ++i) + nested_types.push_back(getInternalType(struct_type->field(i)->type(), tuple_nested_types[i], column_name, format_name)); + + return std::make_shared(std::move(nested_types)); + } + if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), [=](auto && elem) { return elem.first == arrow_type->id(); }); internal_type_it != arrow_type_to_internal_type.end()) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 0e9968bec17..4626d87f0cc 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -6,11 +6,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -113,7 +115,7 @@ namespace DB size_t start, size_t end) { - const auto * column_array = static_cast(column.get()); + const auto * column_array = typeid_cast(column.get()); ColumnPtr nested_column = column_array->getDataPtr(); DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); const auto & offsets = column_array->getOffsets(); @@ -124,13 +126,41 @@ namespace DB for (size_t array_idx = start; array_idx < end; ++array_idx) { - /// Start new array + /// Start new array. components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx]); } } + static void fillArrowArrayWithTupleColumnData( + const String & column_name, + ColumnPtr & column, + const std::shared_ptr & column_type, + const PaddedPODArray * null_bytemap, + arrow::ArrayBuilder * array_builder, + String format_name, + size_t start, + size_t end) + { + const auto * column_tuple = typeid_cast(column.get()); + const auto & nested_types = typeid_cast(column_type.get())->getElements(); + + arrow::StructBuilder & builder = assert_cast(*array_builder); + + for (size_t i = 0; i != column_tuple->tupleSize(); ++i) + { + ColumnPtr nested_column = column_tuple->getColumnPtr(i); + fillArrowArray(column_name + "." + std::to_string(i), nested_column, nested_types[i], null_bytemap, builder.field_builder(i), format_name, start, end); + } + + for (size_t i = start; i != end; ++i) + { + auto status = builder.Append(); + checkStatus(status, column->getName(), format_name); + } + } + template static void fillArrowArrayWithStringColumnData( ColumnPtr write_column, @@ -251,6 +281,10 @@ namespace DB { fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end); } + else if ("Tuple" == column_type_name) + { + fillArrowArrayWithTupleColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end); + } else if (isDecimal(column_type)) { auto fill_decimal = [&](const auto & types) -> bool @@ -351,6 +385,19 @@ namespace DB return arrow::list(nested_arrow_type); } + if (isTuple(column_type)) + { + const auto & nested_types = typeid_cast(column_type.get())->getElements(); + std::vector> nested_fields; + for (size_t i = 0; i != nested_types.size(); ++i) + { + String name = column_name + "." + std::to_string(i); + auto nested_arrow_type = getArrowType(nested_types[i], name, format_name, is_column_nullable); + nested_fields.push_back(std::make_shared(name, nested_arrow_type, *is_column_nullable)); + } + return arrow::struct_(std::move(nested_fields)); + } + const std::string type_name = column_type->getFamilyName(); if (const auto * arrow_type_it = std::find_if( internal_type_to_arrow_type.begin(), diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 24b524faeaf..572b1c130e3 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -67,7 +67,7 @@ void ORCBlockInputFormat::resetParser() stripe_current = 0; } -size_t countIndicesForType(std::shared_ptr type) +static size_t countIndicesForType(std::shared_ptr type) { if (type->id() == arrow::Type::LIST) return countIndicesForType(static_cast(type.get())->value_type()) + 1; diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index ec6a7a65573..fa6f386adf1 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -10,12 +10,14 @@ #include #include #include +#include #include #include #include #include #include +#include namespace DB { @@ -46,15 +48,9 @@ void ORCOutputStream::write(const void* buf, size_t length) ORCBlockOutputFormat::ORCBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) : IOutputFormat(header_, out_), format_settings{format_settings_}, output_stream(out_), data_types(header_.getDataTypes()) { - schema = orc::createStructType(); - options.setCompression(orc::CompressionKind::CompressionKind_NONE); - size_t columns_count = header_.columns(); - for (size_t i = 0; i != columns_count; ++i) - schema->addStructField(header_.safeGetByPosition(i).name, getORCType(data_types[i])); - writer = orc::createWriter(*schema, &output_stream, options); } -ORC_UNIQUE_PTR ORCBlockOutputFormat::getORCType(const DataTypePtr & type) +ORC_UNIQUE_PTR ORCBlockOutputFormat::getORCType(const DataTypePtr & type, const std::string & column_name) { switch (type->getTypeId()) { @@ -102,12 +98,12 @@ ORC_UNIQUE_PTR ORCBlockOutputFormat::getORCType(const DataTypePtr & t } case TypeIndex::Nullable: { - return getORCType(removeNullable(type)); + return getORCType(removeNullable(type), column_name); } case TypeIndex::Array: { const auto * array_type = typeid_cast(type.get()); - return orc::createListType(getORCType(array_type->getNestedType())); + return orc::createListType(getORCType(array_type->getNestedType(), column_name)); } case TypeIndex::Decimal32: { @@ -124,6 +120,18 @@ ORC_UNIQUE_PTR ORCBlockOutputFormat::getORCType(const DataTypePtr & t const auto * decimal_type = typeid_cast *>(type.get()); return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale()); } + case TypeIndex::Tuple: + { + const auto * tuple_type = typeid_cast(type.get()); + const auto & nested_types = tuple_type->getElements(); + auto struct_type = orc::createStructType(); + for (size_t i = 0; i < nested_types.size(); ++i) + { + String name = column_name + "." + std::to_string(i); + struct_type->addStructField(name, getORCType(nested_types[i], name)); + } + return struct_type; + } default: { throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN); @@ -149,6 +157,8 @@ void ORCBlockOutputFormat::writeNumbers( number_orc_column.notNull[i] = 0; continue; } + else + number_orc_column.notNull[i] = 1; number_orc_column.data[i] = convert(number_column.getElement(i)); } number_orc_column.numElements = number_column.size(); @@ -175,6 +185,9 @@ void ORCBlockOutputFormat::writeDecimals( decimal_orc_column.notNull[i] = 0; continue; } + else + decimal_orc_column.notNull[i] = 1; + decimal_orc_column.values[i] = convert(decimal_column.getElement(i).value); } decimal_orc_column.numElements = decimal_column.size(); @@ -197,6 +210,9 @@ void ORCBlockOutputFormat::writeStrings( string_orc_column.notNull[i] = 0; continue; } + else + string_orc_column.notNull[i] = 1; + const StringRef & string = string_column.getDataAt(i); string_orc_column.data[i] = const_cast(string.data); string_orc_column.length[i] = string.size; @@ -223,6 +239,9 @@ void ORCBlockOutputFormat::writeDateTimes( timestamp_orc_column.notNull[i] = 0; continue; } + else + timestamp_orc_column.notNull[i] = 1; + timestamp_orc_column.data[i] = get_seconds(timestamp_column.getElement(i)); timestamp_orc_column.nanoseconds[i] = get_nanoseconds(timestamp_column.getElement(i)); } @@ -235,11 +254,10 @@ void ORCBlockOutputFormat::writeColumn( DataTypePtr & type, const PaddedPODArray * null_bytemap) { + orc_column.notNull.resize(column.size()); if (null_bytemap) - { orc_column.hasNulls = true; - orc_column.notNull.resize(column.size()); - } + switch (type->getTypeId()) { case TypeIndex::Int8: @@ -374,12 +392,25 @@ void ORCBlockOutputFormat::writeColumn( for (size_t i = 0; i != list_column.size(); ++i) { list_orc_column.offsets[i + 1] = offsets[i]; + list_orc_column.notNull[i] = 1; } orc::ColumnVectorBatch & nested_orc_column = *list_orc_column.elements; writeColumn(nested_orc_column, list_column.getData(), nested_type, null_bytemap); list_orc_column.numElements = list_column.size(); break; } + case TypeIndex::Tuple: + { + orc::StructVectorBatch & struct_orc_column = dynamic_cast(orc_column); + const auto & tuple_column = assert_cast(column); + auto nested_types = typeid_cast(type.get())->getElements(); + for (size_t i = 0; i != tuple_column.size(); ++i) + struct_orc_column.notNull[i] = 1; + for (size_t i = 0; i != tuple_column.tupleSize(); ++i) + writeColumn(*struct_orc_column.fields[i], tuple_column.getColumn(i), nested_types[i], null_bytemap); + break; + + } default: throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN); } @@ -409,6 +440,8 @@ size_t ORCBlockOutputFormat::getMaxColumnSize(Chunk & chunk) void ORCBlockOutputFormat::consume(Chunk chunk) { + if (!writer) + prepareWriter(); size_t columns_num = chunk.getNumColumns(); size_t rows_num = chunk.getNumRows(); /// getMaxColumnSize is needed to write arrays. @@ -425,9 +458,23 @@ void ORCBlockOutputFormat::consume(Chunk chunk) void ORCBlockOutputFormat::finalize() { + if (!writer) + prepareWriter(); + writer->close(); } +void ORCBlockOutputFormat::prepareWriter() +{ + const Block & header = getPort(PortKind::Main).getHeader(); + schema = orc::createStructType(); + options.setCompression(orc::CompressionKind::CompressionKind_NONE); + size_t columns_count = header.columns(); + for (size_t i = 0; i != columns_count; ++i) + schema->addStructField(header.safeGetByPosition(i).name, getORCType(data_types[i], header.safeGetByPosition(i).name)); + writer = orc::createWriter(*schema, &output_stream, options); +} + void registerOutputFormatProcessorORC(FormatFactory & factory) { factory.registerOutputFormatProcessor("ORC", []( diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index 05053317533..557bf6cc07a 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -43,7 +43,7 @@ public: void finalize() override; private: - ORC_UNIQUE_PTR getORCType(const DataTypePtr & type); + ORC_UNIQUE_PTR getORCType(const DataTypePtr & type, const std::string & column_name); /// ConvertFunc is needed for type UInt8, because firstly UInt8 (char8_t) must be /// converted to unsigned char (bugprone-signed-char-misuse in clang). @@ -71,6 +71,8 @@ private: size_t getColumnSize(const IColumn & column, DataTypePtr & type); size_t getMaxColumnSize(Chunk & chunk); + void prepareWriter(); + const FormatSettings format_settings; ORCOutputStream output_stream; DataTypes data_types; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index df264406cfe..969aaf6ff36 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -67,6 +67,24 @@ void ParquetBlockInputFormat::resetParser() row_group_current = 0; } +static size_t countIndicesForType(std::shared_ptr type) +{ + if (type->id() == arrow::Type::LIST) + return countIndicesForType(static_cast(type.get())->value_type()); + + int indices = 0; + if (type->id() == arrow::Type::STRUCT) + { + auto * struct_type = static_cast(type.get()); + for (int i = 0; i != struct_type->num_fields(); ++i) + indices += countIndicesForType(struct_type->field(i)->type()); + } + else + indices = 1; + + return indices; +} + void ParquetBlockInputFormat::prepareReader() { THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(in), arrow::default_memory_pool(), &file_reader)); @@ -76,11 +94,14 @@ void ParquetBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); + int index = 0; for (int i = 0; i < schema->num_fields(); ++i) { if (getPort().getHeader().has(schema->field(i)->name())) { - column_indices.push_back(i); + int indexes_count = countIndicesForType(schema->field(i)->type()); + for (int j = 0; j != indexes_count; ++j) + column_indices.push_back(index++); } } } diff --git a/tests/queries/0_stateless/00900_orc_tuples_load.reference b/tests/queries/0_stateless/00900_orc_tuples_load.reference new file mode 100644 index 00000000000..cbf8ee819f7 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_tuples_load.reference @@ -0,0 +1,2 @@ +(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] +(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] diff --git a/tests/queries/0_stateless/00900_orc_tuples_load.sh b/tests/queries/0_stateless/00900_orc_tuples_load.sh new file mode 100755 index 00000000000..f32be6af751 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_tuples_load.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_tuples" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_tuples FORMAT ORC" > "${CLICKHOUSE_TMP}"/tuples.orc + +cat "${CLICKHOUSE_TMP}"/tuples.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_tuples FORMAT ORC" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_tuples" +${CLICKHOUSE_CLIENT} --query="DROP TABLE orc_tuples" diff --git a/tests/queries/0_stateless/00900_parquet_tuples_load.reference b/tests/queries/0_stateless/00900_parquet_tuples_load.reference new file mode 100644 index 00000000000..cbf8ee819f7 --- /dev/null +++ b/tests/queries/0_stateless/00900_parquet_tuples_load.reference @@ -0,0 +1,2 @@ +(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] +(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] diff --git a/tests/queries/0_stateless/00900_parquet_tuples_load.sh b/tests/queries/0_stateless/00900_parquet_tuples_load.sh new file mode 100755 index 00000000000..031e8111019 --- /dev/null +++ b/tests/queries/0_stateless/00900_parquet_tuples_load.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_tuples" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_tuples FORMAT Parquet" > "${CLICKHOUSE_TMP}"/tuples.parquet + +cat "${CLICKHOUSE_TMP}"/tuples.parquet | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_tuples FORMAT Parquet" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_tuples" +${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_tuples" diff --git a/tests/queries/0_stateless/01273_arrow_arrays_load.sh b/tests/queries/0_stateless/01273_arrow_arrays_load.sh index b8d1a85921d..bce653376a5 100755 --- a/tests/queries/0_stateless/01273_arrow_arrays_load.sh +++ b/tests/queries/0_stateless/01273_arrow_arrays_load.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_arrays" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_arrays" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_arrays (arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime)) ENGINE=Memory()" ${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_arrays VALUES ([1,-2,3],[1,2,3],[100,-200,300],[100,200,300],[10000000,-20000000,30000000],[10000000,2000000,3000000],[100000000000000,-200000000000,3000000000000],[100000000000000,20000000000000,3000000000000],['Some string','Some string','Some string'],['0000','1111','2222'],[42.42,424.2,0.4242],[424242.424242,4242042420.242424,42],['2000-01-01','2001-01-01','2002-01-01'],['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00']),([],[],[],[],[],[],[],[],[],[],[],[],[],[])" diff --git a/tests/queries/0_stateless/01273_arrow_tuples_load.reference b/tests/queries/0_stateless/01273_arrow_tuples_load.reference new file mode 100644 index 00000000000..cbf8ee819f7 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_tuples_load.reference @@ -0,0 +1,2 @@ +(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] +(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] diff --git a/tests/queries/0_stateless/01273_arrow_tuples_load.sh b/tests/queries/0_stateless/01273_arrow_tuples_load.sh new file mode 100755 index 00000000000..311079afe50 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_tuples_load.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_tuples" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_tuples FORMAT Arrow" > "${CLICKHOUSE_TMP}"/tuples.arrow + +cat "${CLICKHOUSE_TMP}"/tuples.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_tuples FORMAT Arrow" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_tuples" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_tuples" From d74c5ef62d02a91028d4f4152b358c2c837515a2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 20 May 2021 17:02:44 +0300 Subject: [PATCH 03/43] Minor change --- .../Formats/Impl/ORCBlockOutputFormat.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index fa6f386adf1..a560701944d 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -157,8 +157,8 @@ void ORCBlockOutputFormat::writeNumbers( number_orc_column.notNull[i] = 0; continue; } - else - number_orc_column.notNull[i] = 1; + + number_orc_column.notNull[i] = 1; number_orc_column.data[i] = convert(number_column.getElement(i)); } number_orc_column.numElements = number_column.size(); @@ -185,9 +185,8 @@ void ORCBlockOutputFormat::writeDecimals( decimal_orc_column.notNull[i] = 0; continue; } - else - decimal_orc_column.notNull[i] = 1; + decimal_orc_column.notNull[i] = 1; decimal_orc_column.values[i] = convert(decimal_column.getElement(i).value); } decimal_orc_column.numElements = decimal_column.size(); @@ -210,9 +209,8 @@ void ORCBlockOutputFormat::writeStrings( string_orc_column.notNull[i] = 0; continue; } - else - string_orc_column.notNull[i] = 1; + string_orc_column.notNull[i] = 1; const StringRef & string = string_column.getDataAt(i); string_orc_column.data[i] = const_cast(string.data); string_orc_column.length[i] = string.size; @@ -239,9 +237,8 @@ void ORCBlockOutputFormat::writeDateTimes( timestamp_orc_column.notNull[i] = 0; continue; } - else - timestamp_orc_column.notNull[i] = 1; + timestamp_orc_column.notNull[i] = 1; timestamp_orc_column.data[i] = get_seconds(timestamp_column.getElement(i)); timestamp_orc_column.nanoseconds[i] = get_nanoseconds(timestamp_column.getElement(i)); } From 540c494492add6999fdb3e956fd00fdc821602d0 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 24 May 2021 14:39:08 +0300 Subject: [PATCH 04/43] Fix selecting indexes in ORC and Parquet formats --- src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 13 ++++++++----- .../Formats/Impl/ParquetBlockInputFormat.cpp | 8 ++++++-- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 572b1c130e3..d16d0870474 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -84,17 +84,20 @@ void ORCBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); - int index = 0; + /// In ReadStripe column indices should be started from 1, + /// because 0 indicates to select all columns. + int index = 1; for (int i = 0; i < schema->num_fields(); ++i) { + /// LIST type require 2 indices, so we should recursively + /// count the number of indices we need for this type. + int indexes_count = countIndicesForType(schema->field(i)->type()); if (getPort().getHeader().has(schema->field(i)->name())) { - /// LIST type require 2 indices, so we should recursively - /// count the number of indices we need for this type. - int indexes_count = countIndicesForType(schema->field(i)->type()); for (int j = 0; j != indexes_count; ++j) - include_indices.push_back(index++); + include_indices.push_back(index + j); } + index += indexes_count; } } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 969aaf6ff36..0223e5219c0 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -97,12 +97,16 @@ void ParquetBlockInputFormat::prepareReader() int index = 0; for (int i = 0; i < schema->num_fields(); ++i) { + /// STRUCT type require the number of indexes equal to the number of + /// nested elements, so we should recursively + /// count the number of indices we need for this type. + int indexes_count = countIndicesForType(schema->field(i)->type()); if (getPort().getHeader().has(schema->field(i)->name())) { - int indexes_count = countIndicesForType(schema->field(i)->type()); for (int j = 0; j != indexes_count; ++j) - column_indices.push_back(index++); + column_indices.push_back(index + j); } + index += indexes_count; } } From c8b37977da865ad7c3cd0f22b9a9f5a0944ab6d6 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 25 May 2021 15:01:28 +0300 Subject: [PATCH 05/43] Fix bugs, support dictionary for Arrow format --- src/Columns/ColumnLowCardinality.h | 1 + src/Core/Settings.h | 5 +- src/DataTypes/DataTypeLowCardinality.h | 2 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/ArrowBlockInputFormat.cpp | 2 +- .../Formats/Impl/ArrowBlockInputFormat.h | 3 + .../Formats/Impl/ArrowBlockOutputFormat.cpp | 2 +- .../Formats/Impl/ArrowBlockOutputFormat.h | 2 + .../Formats/Impl/ArrowColumnToCHColumn.cpp | 77 ++++++- .../Formats/Impl/ArrowColumnToCHColumn.h | 16 +- .../Formats/Impl/CHColumnToArrowColumn.cpp | 197 ++++++++++++++++-- .../Formats/Impl/CHColumnToArrowColumn.h | 21 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 16 +- .../Formats/Impl/ORCBlockInputFormat.h | 3 + .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- .../Formats/Impl/ParquetBlockInputFormat.h | 2 + .../Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.h | 2 + .../01273_arrow_dictionaries_load.reference | 4 + .../01273_arrow_dictionaries_load.sh | 24 +++ 21 files changed, 346 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/01273_arrow_dictionaries_load.reference create mode 100755 tests/queries/0_stateless/01273_arrow_dictionaries_load.sh diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index fc607021ccf..b39e4cf79c0 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -190,6 +190,7 @@ public: void nestedToNullable() { dictionary.getColumnUnique().nestedToNullable(); } const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } + IColumnUnique & getDictionary() { return dictionary.getColumnUnique(); } const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); } /// IColumnUnique & getUnique() { return static_cast(*column_unique); } /// ColumnPtr getUniquePtr() const { return column_unique; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2aed174c088..4815f060140 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -117,7 +117,7 @@ class IColumn; M(Bool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ - M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \ + M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards. If 2 - same as 1 but also apply ORDER BY and LIMIT stages", 0) \ M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ @@ -559,7 +559,8 @@ class IColumn; M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ M(Bool, cross_to_inner_join_rewrite, true, "Use inner join instead of comma/cross join if possible", 0) \ - + \ + M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 1266174c6d6..2ab62fabf41 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -13,7 +13,7 @@ private: DataTypePtr dictionary_type; public: - DataTypeLowCardinality(DataTypePtr dictionary_type_); + DataTypeLowCardinality(DataTypePtr dictionary_type_dict); const DataTypePtr & getDictionaryType() const { return dictionary_type; } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f1f60ae2ac4..8b7cf9635b4 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -112,6 +112,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; format_settings.with_names_use_header = settings.input_format_with_names_use_header; format_settings.write_statistics = settings.output_format_write_statistics; + format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 33d51b1797f..1773f2cc2c6 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -52,6 +52,7 @@ struct FormatSettings struct { UInt64 row_group_size = 1000000; + bool low_cardinality_as_dictionary = false; } arrow; struct diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 52d2cf98c25..0ac8251b8bb 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -63,7 +63,7 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; - ArrowColumnToCHColumn::arrowTableToCHChunk(res, *table_result, header, "Arrow"); + arrow_column_to_ch_column.arrowTableToCHChunk(res, *table_result, header, "Arrow"); return res; } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 5ad112efde9..6a8acc4a118 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -3,6 +3,7 @@ #if USE_ARROW #include +#include namespace arrow { class RecordBatchReader; } namespace arrow::ipc { class RecordBatchFileReader; } @@ -32,6 +33,8 @@ private: // The following fields are used only for Arrow format std::shared_ptr file_reader; + ArrowColumnToCHColumn arrow_column_to_ch_column; + int record_batch_total = 0; int record_batch_current = 0; diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index c1abdd1a759..d1fdffb700c 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -28,7 +28,7 @@ void ArrowBlockOutputFormat::consume(Chunk chunk) const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; - CHColumnToArrowColumn::chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow"); + ch_column_to_arrow_column.chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow", format_settings.arrow.low_cardinality_as_dictionary); if (!writer) prepareWriter(arrow_table->schema()); diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h index 0cc6804705b..fc8efe62435 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h @@ -4,6 +4,7 @@ #include #include +#include #include "ArrowBufferedStreams.h" namespace arrow { class Schema; } @@ -28,6 +29,7 @@ private: const FormatSettings format_settings; std::shared_ptr arrow_ostream; std::shared_ptr writer; + CHColumnToArrowColumn ch_column_to_arrow_column; void prepareWriter(const std::shared_ptr & schema); }; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 23f356a77fc..23987067474 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -16,6 +16,8 @@ #include #include #include +#include +#include #include #include @@ -266,12 +268,33 @@ namespace DB } } - static void readColumnFromArrowColumn(std::shared_ptr & arrow_column, IColumn & internal_column, const std::string & column_name, const std::string format_name, bool is_nullable) + static DataTypePtr getInternalIndexesType(std::shared_ptr arrow_type) + { + switch (arrow_type->id()) + { +# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ + case ARROW_NUMERIC_TYPE: \ + return std::make_shared>(); \ + + FOR_ARROW_INXEXES_TYPES(DISPATCH) +# undef DISPATCH + default: + throw Exception("Unsupported type for indexes in LowCardinality: " + arrow_type->name() + ".", ErrorCodes::BAD_ARGUMENTS); + } + } + + static void readColumnFromArrowColumn( + std::shared_ptr & arrow_column, + IColumn & internal_column, + const std::string & column_name, + const std::string format_name, + bool is_nullable, + std::unordered_map dictionary_values) { if (internal_column.isNullable()) { ColumnNullable & column_nullable = typeid_cast(internal_column); - readColumnFromArrowColumn(arrow_column, column_nullable.getNestedColumn(), column_name, format_name, true); + readColumnFromArrowColumn(arrow_column, column_nullable.getNestedColumn(), column_name, format_name, true, dictionary_values); fillByteMapFromArrowColumn(arrow_column, column_nullable.getNullMapColumn()); return; } @@ -322,7 +345,7 @@ namespace DB auto arrow_nested_column = std::make_shared(array_vector); ColumnArray & column_array = typeid_cast(internal_column); - readColumnFromArrowColumn(arrow_nested_column, column_array.getData(), column_name, format_name, false); + readColumnFromArrowColumn(arrow_nested_column, column_array.getData(), column_name, format_name, false, dictionary_values); fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn()); break; } @@ -341,10 +364,48 @@ namespace DB for (int i = 0; i != fields_count; ++i) { auto nested_arrow_column = std::make_shared(nested_arrow_columns[i]); - readColumnFromArrowColumn(nested_arrow_column, column_tuple.getColumn(i), column_name, format_name, false); + readColumnFromArrowColumn(nested_arrow_column, column_tuple.getColumn(i), column_name, format_name, false, dictionary_values); } break; } + case arrow::Type::DICTIONARY: + { + ColumnLowCardinality & column_lc = typeid_cast(internal_column); + auto & dict_values = dictionary_values[column_name]; + /// Load dictionary values only once and reuse it. + if (!dict_values) + { + arrow::ArrayVector dict_array; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::DictionaryArray & dict_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + dict_array.emplace_back(dict_chunk.dictionary()); + } + auto arrow_dict_column = std::make_shared(dict_array); + + auto dict_column = IColumn::mutate(column_lc.getDictionaryPtr()); + auto * uniq_column = static_cast(dict_column.get()); + auto values_column = uniq_column->getNestedColumn()->cloneEmpty(); + readColumnFromArrowColumn(arrow_dict_column, *values_column, column_name, format_name, false, dictionary_values); + uniq_column->uniqueInsertRangeFrom(*values_column, 0, values_column->size()); + dict_values = std::move(dict_column); + } + + arrow::ArrayVector indexes_array; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::DictionaryArray & dict_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + indexes_array.emplace_back(dict_chunk.indices()); + } + + auto arrow_indexes_column = std::make_shared(indexes_array); + auto indexes_column = getInternalIndexesType(arrow_indexes_column->type())->createColumn(); + + readColumnFromArrowColumn(arrow_indexes_column, *indexes_column, column_name, format_name, is_nullable, dictionary_values); + auto new_column_lc = ColumnLowCardinality::create(dict_values, std::move(indexes_column)); + column_lc = std::move(*new_column_lc); + break; + } # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ fillColumnWithNumericData(arrow_column, internal_column); \ @@ -414,6 +475,12 @@ namespace DB return std::make_shared(std::move(nested_types)); } + if (arrow_type->id() == arrow::Type::DICTIONARY) + { + const auto * arrow_dict_type = static_cast(arrow_type.get()); + return std::make_shared(getInternalType(arrow_dict_type->value_type(), column_type, column_name, format_name)); + } + if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), [=](auto && elem) { return elem.first == arrow_type->id(); }); internal_type_it != arrow_type_to_internal_type.end()) @@ -459,7 +526,7 @@ namespace DB DataTypePtr internal_type = getInternalType(arrow_column->type(), column_type, header_column.name, format_name); MutableColumnPtr read_column = internal_type->createColumn(); - readColumnFromArrowColumn(arrow_column, *read_column, header_column.name, format_name, false); + readColumnFromArrowColumn(arrow_column, *read_column, header_column.name, format_name, false, dictionary_values); ColumnWithTypeAndName column; column.name = header_column.name; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index abac501c4c5..60fbc30e9c9 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -36,10 +36,24 @@ namespace DB M(arrow::Type::FLOAT, DB::Float32) \ M(arrow::Type::DOUBLE, DB::Float64) +# define FOR_ARROW_INXEXES_TYPES(M) \ + M(arrow::Type::UINT8, DB::UInt8) \ + M(arrow::Type::INT8, DB::UInt8) \ + M(arrow::Type::UINT16, DB::UInt16) \ + M(arrow::Type::INT16, DB::UInt16) \ + M(arrow::Type::UINT32, DB::UInt32) \ + M(arrow::Type::INT32, DB::UInt32) \ + M(arrow::Type::UINT64, DB::UInt64) \ + M(arrow::Type::INT64, DB::UInt64) \ + + /// Map {column name : dictionary column}. + /// To avoid converting dictionary from Arrow Dictionary + /// to LowCardinality every chunk we save it and reuse. + std::unordered_map dictionary_values; public: - static void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, + void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, const Block & header, std::string format_name); }; } diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 4626d87f0cc..97a5326a421 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -7,18 +7,19 @@ #include #include #include +#include #include #include #include #include #include #include +#include #include #include #include #include #include -#include namespace DB @@ -103,7 +104,8 @@ namespace DB arrow::ArrayBuilder * array_builder, String format_name, size_t start, - size_t end); + size_t end, + std::unordered_map> & dictionary_values); static void fillArrowArrayWithArrayColumnData( const String & column_name, @@ -113,7 +115,8 @@ namespace DB arrow::ArrayBuilder * array_builder, String format_name, size_t start, - size_t end) + size_t end, + std::unordered_map> & dictionary_values) { const auto * column_array = typeid_cast(column.get()); ColumnPtr nested_column = column_array->getDataPtr(); @@ -129,7 +132,7 @@ namespace DB /// Start new array. components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); - fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx]); + fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx], dictionary_values); } } @@ -141,7 +144,8 @@ namespace DB arrow::ArrayBuilder * array_builder, String format_name, size_t start, - size_t end) + size_t end, + std::unordered_map> & dictionary_values) { const auto * column_tuple = typeid_cast(column.get()); const auto & nested_types = typeid_cast(column_type.get())->getElements(); @@ -151,7 +155,7 @@ namespace DB for (size_t i = 0; i != column_tuple->tupleSize(); ++i) { ColumnPtr nested_column = column_tuple->getColumnPtr(i); - fillArrowArray(column_name + "." + std::to_string(i), nested_column, nested_types[i], null_bytemap, builder.field_builder(i), format_name, start, end); + fillArrowArray(column_name + "." + std::to_string(i), nested_column, nested_types[i], null_bytemap, builder.field_builder(i), format_name, start, end, dictionary_values); } for (size_t i = start; i != end; ++i) @@ -161,6 +165,115 @@ namespace DB } } + template + static PaddedPODArray extractIndexesImpl(ColumnPtr column, size_t start, size_t end) + { + const PaddedPODArray & data = checkAndGetColumn>(*column)->getData(); + PaddedPODArray result; + result.reserve(end - start); + std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value); }); + return result; + } + + static PaddedPODArray extractIndexesImpl(ColumnPtr column, size_t start, size_t end) + { + switch (column->getDataType()) + { + case TypeIndex::UInt8: + return extractIndexesImpl(column, start, end); + case TypeIndex::UInt16: + return extractIndexesImpl(column, start, end); + case TypeIndex::UInt32: + return extractIndexesImpl(column, start, end); + case TypeIndex::UInt64: + return extractIndexesImpl(column, start, end); + default: + throw Exception("Indexes column must be ColumnUInt, got " + column->getName(), + ErrorCodes::LOGICAL_ERROR); + } + } + + template + static void fillArrowArrayWithLowCardinalityColumnDataImpl( + const String & column_name, + ColumnPtr & column, + const std::shared_ptr & column_type, + const PaddedPODArray * null_bytemap, + arrow::ArrayBuilder * array_builder, + String format_name, + size_t start, + size_t end, + std::unordered_map> & dictionary_values) + { + const auto * column_lc = typeid_cast(column.get()); + arrow::DictionaryBuilder * builder = typeid_cast *>(array_builder); + auto & dict_values = dictionary_values[column_name]; + + /// Convert dictionary from LowCardinality to Arrow dictionary only once and then reuse it. + if (!dict_values) + { + auto value_type = typeid_cast(builder->type().get())->value_type(); + std::unique_ptr values_builder; + arrow::MemoryPool* pool = arrow::default_memory_pool(); + arrow::Status status = MakeBuilder(pool, value_type, &values_builder); + checkStatus(status, column->getName(), format_name); + + auto dict_column = column_lc->getDictionary().getNestedColumn(); + const auto & dict_type = typeid_cast(column_type.get())->getDictionaryType(); + fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, 0, dict_column->size(), dictionary_values); + status = values_builder->Finish(&dict_values); + checkStatus(status, column->getName(), format_name); + } + + arrow::Status status = builder->InsertMemoValues(*dict_values); + checkStatus(status, column->getName(), format_name); + + /// AppendIndices in DictionaryBuilder works only with int64_t data, so we cannot use + /// fillArrowArray here and should copy all indexes to int64_t container. + auto indexes = extractIndexesImpl(column_lc->getIndexesPtr(), start, end); + const uint8_t * arrow_null_bytemap_raw_ptr = nullptr; + PaddedPODArray arrow_null_bytemap; + if (null_bytemap) + { + /// Invert values since Arrow interprets 1 as a non-null value, while CH as a null + arrow_null_bytemap.reserve(end - start); + for (size_t i = start; i < end; ++i) + arrow_null_bytemap.emplace_back(!(*null_bytemap)[i]); + + arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data(); + } + + status = builder->AppendIndices(indexes.data(), indexes.size(), arrow_null_bytemap_raw_ptr); + checkStatus(status, column->getName(), format_name); + } + + + static void fillArrowArrayWithLowCardinalityColumnData( + const String & column_name, + ColumnPtr & column, + const std::shared_ptr & column_type, + const PaddedPODArray * null_bytemap, + arrow::ArrayBuilder * array_builder, + String format_name, + size_t start, + size_t end, + std::unordered_map> & dictionary_values) + { + auto * dict_type = typeid_cast(array_builder->type().get()); + auto value_type = dict_type->value_type(); + +#define DISPATCH(ARROW_TYPE_ID, ARROW_TYPE) \ + if (arrow::Type::ARROW_TYPE_ID == value_type->id()) \ + { \ + fillArrowArrayWithLowCardinalityColumnDataImpl(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); \ + return; \ + } + + FOR_ARROW_TYPES(DISPATCH) +#undef DISPATCH + + } + template static void fillArrowArrayWithStringColumnData( ColumnPtr write_column, @@ -185,7 +298,6 @@ namespace DB StringRef string_ref = internal_column.getDataAt(string_i); status = builder.Append(string_ref.data, string_ref.size); } - checkStatus(status, write_column->getName(), format_name); } } @@ -248,7 +360,8 @@ namespace DB arrow::ArrayBuilder * array_builder, String format_name, size_t start, - size_t end) + size_t end, + std::unordered_map> & dictionary_values) { const String column_type_name = column_type->getFamilyName(); @@ -259,7 +372,7 @@ namespace DB DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); const ColumnPtr & null_column = column_nullable->getNullMapColumnPtr(); const PaddedPODArray & bytemap = assert_cast &>(*null_column).getData(); - fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end); + fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end, dictionary_values); } else if ("String" == column_type_name) { @@ -279,11 +392,15 @@ namespace DB } else if ("Array" == column_type_name) { - fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end); + fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); } else if ("Tuple" == column_type_name) { - fillArrowArrayWithTupleColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end); + fillArrowArrayWithTupleColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); + } + else if ("LowCardinality" == column_type_name) + { + fillArrowArrayWithLowCardinalityColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); } else if (isDecimal(column_type)) { @@ -346,12 +463,33 @@ namespace DB checkStatus(status, write_column->getName(), format_name); } - static std::shared_ptr getArrowType(DataTypePtr column_type, const std::string & column_name, const std::string & format_name, bool * is_column_nullable) + static std::shared_ptr getArrowTypeForLowCardinalityIndexes(ColumnPtr indexes_column) + { + /// Arrow docs recommend preferring signed integers over unsigned integers for representing dictionary indices. + /// https://arrow.apache.org/docs/format/Columnar.html#dictionary-encoded-layout + switch (indexes_column->getDataType()) + { + case TypeIndex::UInt8: + return arrow::int8(); + case TypeIndex::UInt16: + return arrow::int16(); + case TypeIndex::UInt32: + return arrow::int32(); + case TypeIndex::UInt64: + return arrow::int64(); + default: + throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got " + indexes_column->getName(), + ErrorCodes::LOGICAL_ERROR); + } + } + + static std::shared_ptr getArrowType(DataTypePtr column_type, ColumnPtr column, const std::string & column_name, const std::string & format_name, bool * is_column_nullable) { if (column_type->isNullable()) { DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); - auto arrow_type = getArrowType(nested_type, column_name, format_name, is_column_nullable); + ColumnPtr nested_column = checkAndGetColumn(*column)->getNestedColumnPtr(); + auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable); *is_column_nullable = true; return arrow_type; } @@ -381,23 +519,36 @@ namespace DB if (isArray(column_type)) { auto nested_type = typeid_cast(column_type.get())->getNestedType(); - auto nested_arrow_type = getArrowType(nested_type, column_name, format_name, is_column_nullable); + auto nested_column = checkAndGetColumn(*column)->getDataPtr(); + auto nested_arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable); return arrow::list(nested_arrow_type); } if (isTuple(column_type)) { const auto & nested_types = typeid_cast(column_type.get())->getElements(); + const auto * tuple_column = checkAndGetColumn(*column); std::vector> nested_fields; for (size_t i = 0; i != nested_types.size(); ++i) { String name = column_name + "." + std::to_string(i); - auto nested_arrow_type = getArrowType(nested_types[i], name, format_name, is_column_nullable); + auto nested_arrow_type = getArrowType(nested_types[i], tuple_column->getColumnPtr(i), name, format_name, is_column_nullable); nested_fields.push_back(std::make_shared(name, nested_arrow_type, *is_column_nullable)); } return arrow::struct_(std::move(nested_fields)); } + if (column_type->lowCardinality()) + { + auto nested_type = typeid_cast(column_type.get())->getDictionaryType(); + const auto * lc_column = checkAndGetColumn(*column); + auto nested_column = lc_column->getDictionaryPtr(); + auto indexes_column = lc_column->getIndexesPtr(); + return arrow::dictionary( + getArrowTypeForLowCardinalityIndexes(indexes_column), + getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable)); + } + const std::string type_name = column_type->getFamilyName(); if (const auto * arrow_type_it = std::find_if( internal_type_to_arrow_type.begin(), @@ -418,7 +569,8 @@ namespace DB const Block & header, const Chunk & chunk, size_t columns_num, - String format_name) + String format_name, + bool low_cardinality_as_dictionary) { /// For arrow::Schema and arrow::Table creation std::vector> arrow_fields; @@ -430,11 +582,16 @@ namespace DB { // TODO: constructed every iteration ColumnWithTypeAndName column = header.safeGetByPosition(column_i); - column.column = recursiveRemoveLowCardinality(chunk.getColumns()[column_i]); - column.type = recursiveRemoveLowCardinality(column.type); + column.column = chunk.getColumns()[column_i]; + + if (!low_cardinality_as_dictionary) + { + column.column = recursiveRemoveLowCardinality(column.column); + column.type = recursiveRemoveLowCardinality(column.type); + } bool is_column_nullable = false; - auto arrow_type = getArrowType(column.type, column.name, format_name, &is_column_nullable); + auto arrow_type = getArrowType(column.type, column.column, column.name, format_name, &is_column_nullable); arrow_fields.emplace_back(std::make_shared(column.name, arrow_type, is_column_nullable)); arrow::MemoryPool* pool = arrow::default_memory_pool(); @@ -442,7 +599,7 @@ namespace DB arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder); checkStatus(status, column.column->getName(), format_name); - fillArrowArray(column.name, column.column, column.type, nullptr, array_builder.get(), format_name, 0, column.column->size()); + fillArrowArray(column.name, column.column, column.type, nullptr, array_builder.get(), format_name, 0, column.column->size(), dictionary_values); std::shared_ptr arrow_array; status = array_builder->Finish(&arrow_array); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h index de594389c25..9df70b5bceb 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h @@ -26,10 +26,27 @@ private: M(Float32, arrow::FloatBuilder) \ M(Float64, arrow::DoubleBuilder) +#define FOR_ARROW_TYPES(M) \ + M(UINT8, arrow::UInt8Type) \ + M(INT8, arrow::Int8Type) \ + M(UINT16, arrow::UInt16Type) \ + M(INT16, arrow::Int16Type) \ + M(UINT32, arrow::UInt32Type) \ + M(INT32, arrow::Int32Type) \ + M(UINT64, arrow::UInt64Type) \ + M(INT64, arrow::Int64Type) \ + M(FLOAT, arrow::FloatType) \ + M(DOUBLE, arrow::DoubleType) \ + M(STRING, arrow::StringType) + + /// Map {column name : arrow dictionary}. + /// To avoid converting dictionary from LowCardinality to Arrow + /// Dictionary every chunk we save it and reuse. + std::unordered_map> dictionary_values; public: - static void chChunkToArrowTable(std::shared_ptr & res, const Block & header, const Chunk & chunk, - size_t columns_num, String format_name); + void chChunkToArrowTable(std::shared_ptr & res, const Block & header, const Chunk & chunk, + size_t columns_num, String format_name, bool low_cardinality_as_dictionary = false); }; } #endif diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index d16d0870474..7b1c2af1a52 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -54,7 +54,7 @@ Chunk ORCBlockInputFormat::generate() ++stripe_current; - ArrowColumnToCHColumn::arrowTableToCHChunk(res, *table_result, header, "ORC"); + arrow_column_to_ch_column.arrowTableToCHChunk(res, *table_result, header, "ORC"); return res; } @@ -72,7 +72,15 @@ static size_t countIndicesForType(std::shared_ptr type) if (type->id() == arrow::Type::LIST) return countIndicesForType(static_cast(type.get())->value_type()) + 1; - return 1; + int indices = 1; + if (type->id() == arrow::Type::STRUCT) + { + auto * struct_type = static_cast(type.get()); + for (int i = 0; i != struct_type->num_fields(); ++i) + indices += countIndicesForType(struct_type->field(i)->type()); + } + + return indices; } void ORCBlockInputFormat::prepareReader() @@ -89,8 +97,8 @@ void ORCBlockInputFormat::prepareReader() int index = 1; for (int i = 0; i < schema->num_fields(); ++i) { - /// LIST type require 2 indices, so we should recursively - /// count the number of indices we need for this type. + /// LIST type require 2 indices, STRUCT - the number of elements + 1, + /// so we should recursively count the number of indices we need for this type. int indexes_count = countIndicesForType(schema->field(i)->type()); if (getPort().getHeader().has(schema->field(i)->name())) { diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 0c78290f3cc..5a6cfd1364a 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -3,6 +3,7 @@ #if USE_ORC #include +#include namespace arrow::adapters::orc { class ORCFileReader; } @@ -26,6 +27,8 @@ private: std::unique_ptr file_reader; + ArrowColumnToCHColumn arrow_column_to_ch_column; + int stripe_total = 0; int stripe_current = 0; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 0223e5219c0..1d7feadc1e7 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -54,7 +54,7 @@ Chunk ParquetBlockInputFormat::generate() ++row_group_current; - ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, header, "Parquet"); + arrow_column_to_ch_column.arrowTableToCHChunk(res, table, header, "Parquet"); return res; } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 0841e82d4d0..b27bafe04bf 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -4,6 +4,7 @@ #if USE_PARQUET #include +#include namespace parquet::arrow { class FileReader; } @@ -32,6 +33,7 @@ private: int row_group_total = 0; // indices of columns to read from Parquet file std::vector column_indices; + ArrowColumnToCHColumn arrow_column_to_ch_column; int row_group_current = 0; }; diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 78b6a9c53a4..25eade4bafa 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -36,7 +36,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; - CHColumnToArrowColumn::chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet"); + ch_column_to_arrow_column.chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet"); if (!file_writer) { diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 11d746a0a6d..deb011e0274 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -4,6 +4,7 @@ #if USE_PARQUET # include # include +# include namespace arrow { @@ -36,6 +37,7 @@ private: const FormatSettings format_settings; std::unique_ptr file_writer; + CHColumnToArrowColumn ch_column_to_arrow_column; }; } diff --git a/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference b/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference new file mode 100644 index 00000000000..b144130918f --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference @@ -0,0 +1,4 @@ +1 ['a','b','c'] ('z','6') +2 ['d','e'] ('x','9') +1 ['a','b','c'] ('z','6') +2 ['d','e'] ('x','9') diff --git a/tests/queries/0_stateless/01273_arrow_dictionaries_load.sh b/tests/queries/0_stateless/01273_arrow_dictionaries_load.sh new file mode 100755 index 00000000000..38e6c2c1b01 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_dictionaries_load.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_dicts" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_dicts (a LowCardinality(String), b Array(LowCardinality(String)), c Tuple(LowCardinality(String), LowCardinality(String))) ENGINE=Memory()" +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_dicts VALUES ('1', ['a', 'b', 'c'], ('z', '6')), ('2', ['d', 'e'], ('x', '9'))" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_dicts FORMAT Arrow SETTINGS output_format_arrow_low_cardinality_as_dictionary=1" > "${CLICKHOUSE_TMP}"/dicts.arrow + +cat "${CLICKHOUSE_TMP}"/dicts.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_dicts FORMAT Arrow" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_dicts" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_dicts" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_dicts (a LowCardinality(String)) ENGINE=Memory()" +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_dicts SELECT toString(number % 500) from numbers(10000000)" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_dicts FORMAT Arrow SETTINGS output_format_arrow_low_cardinality_as_dictionary=1" > "${CLICKHOUSE_TMP}"/dicts.arrow + +cat "${CLICKHOUSE_TMP}"/dicts.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_dicts FORMAT Arrow" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_dicts" + From a4decd08484d07928c668dfdfb8bb08eb4f0ba06 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 27 May 2021 22:01:06 +0300 Subject: [PATCH 06/43] Support Map type, fix and add tests --- src/Core/Settings.h | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 41 +++++++--- .../Formats/Impl/CHColumnToArrowColumn.cpp | 77 ++++++++++++------- .../Formats/Impl/ORCBlockInputFormat.cpp | 14 +++- .../Formats/Impl/ORCBlockOutputFormat.cpp | 51 ++++++++++-- .../Formats/Impl/ParquetBlockInputFormat.cpp | 6 ++ .../00900_long_parquet_load.reference | 28 +++++-- .../0_stateless/00900_long_parquet_load.sh | 5 +- .../00900_orc_arrow_parquet_maps.reference | 6 ++ .../00900_orc_arrow_parquet_maps.sh | 28 +++++++ ... 00900_orc_arrow_parquet_tuples.reference} | 4 + .../00900_orc_arrow_parquet_tuples.sh | 24 ++++++ .../queries/0_stateless/00900_orc_map_load.sh | 20 +++++ .../0_stateless/00900_orc_tuples_load.sh | 17 ---- .../00900_parquet_tuples_load.reference | 2 - .../0_stateless/00900_parquet_tuples_load.sh | 17 ---- .../01273_arrow_tuples_load.reference | 2 - .../0_stateless/01273_arrow_tuples_load.sh | 17 ---- .../nested_maps.snappy.parquet.columns | 2 +- .../nonnullable.impala.parquet.columns | 2 +- .../nullable.impala.parquet.columns | 2 +- .../data_parquet/nulls.snappy.parquet.columns | 2 +- 22 files changed, 256 insertions(+), 113 deletions(-) create mode 100644 tests/queries/0_stateless/00900_orc_arrow_parquet_maps.reference create mode 100755 tests/queries/0_stateless/00900_orc_arrow_parquet_maps.sh rename tests/queries/0_stateless/{00900_orc_tuples_load.reference => 00900_orc_arrow_parquet_tuples.reference} (63%) create mode 100755 tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.sh create mode 100644 tests/queries/0_stateless/00900_orc_map_load.sh delete mode 100755 tests/queries/0_stateless/00900_orc_tuples_load.sh delete mode 100644 tests/queries/0_stateless/00900_parquet_tuples_load.reference delete mode 100755 tests/queries/0_stateless/00900_parquet_tuples_load.sh delete mode 100644 tests/queries/0_stateless/01273_arrow_tuples_load.reference delete mode 100755 tests/queries/0_stateless/01273_arrow_tuples_load.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4815f060140..dae186e70ba 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -117,7 +117,7 @@ class IColumn; M(Bool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ - M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards. If 2 - same as 1 but also apply ORDER BY and LIMIT stages", 0) \ + M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \ M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 23987067474..8f76f5dca8a 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -18,6 +19,7 @@ #include #include #include +#include #include #include @@ -293,14 +295,15 @@ namespace DB { if (internal_column.isNullable()) { - ColumnNullable & column_nullable = typeid_cast(internal_column); + ColumnNullable & column_nullable = assert_cast(internal_column); readColumnFromArrowColumn(arrow_column, column_nullable.getNestedColumn(), column_name, format_name, true, dictionary_values); fillByteMapFromArrowColumn(arrow_column, column_nullable.getNullMapColumn()); return; } - // TODO: check if a column is const? - if (!is_nullable && !checkColumn(internal_column) && arrow_column->null_count()) + /// TODO: check if a column is const? + if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST + && arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT) { throw Exception { @@ -332,6 +335,7 @@ namespace DB //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/); break; + case arrow::Type::MAP: [[fallthrough]]; case arrow::Type::LIST: { arrow::ArrayVector array_vector; @@ -344,14 +348,17 @@ namespace DB } auto arrow_nested_column = std::make_shared(array_vector); - ColumnArray & column_array = typeid_cast(internal_column); + ColumnArray & column_array = arrow_column->type()->id() == arrow::Type::MAP + ? assert_cast(internal_column).getNestedColumn() + : assert_cast(internal_column); + readColumnFromArrowColumn(arrow_nested_column, column_array.getData(), column_name, format_name, false, dictionary_values); fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn()); break; } case arrow::Type::STRUCT: { - ColumnTuple & column_tuple = typeid_cast(internal_column); + ColumnTuple & column_tuple = assert_cast(internal_column); int fields_count = column_tuple.tupleSize(); std::vector nested_arrow_columns(fields_count); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -370,7 +377,7 @@ namespace DB } case arrow::Type::DICTIONARY: { - ColumnLowCardinality & column_lc = typeid_cast(internal_column); + ColumnLowCardinality & column_lc = assert_cast(internal_column); auto & dict_values = dictionary_values[column_name]; /// Load dictionary values only once and reuse it. if (!dict_values) @@ -430,7 +437,7 @@ namespace DB { if (column_type->isNullable()) { - DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); + DataTypePtr nested_type = assert_cast(column_type.get())->getNestedType(); return makeNullable(getInternalType(arrow_type, nested_type, column_name, format_name)); } @@ -447,7 +454,7 @@ namespace DB const DataTypeArray * array_type = typeid_cast(column_type.get()); if (!array_type) - throw Exception{"Cannot convert arrow LIST type to a not Array ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + throw Exception{"Cannot convert arrow LIST type to a not Array/Map ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; return std::make_shared(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name)); } @@ -478,7 +485,23 @@ namespace DB if (arrow_type->id() == arrow::Type::DICTIONARY) { const auto * arrow_dict_type = static_cast(arrow_type.get()); - return std::make_shared(getInternalType(arrow_dict_type->value_type(), column_type, column_name, format_name)); + const auto * lc_type = typeid_cast(column_type.get()); + /// We allow to insert arrow dictionary into a non-LowCardinality column. + const auto & dict_type = lc_type ? lc_type->getDictionaryType() : column_type; + return std::make_shared(getInternalType(arrow_dict_type->value_type(), dict_type, column_name, format_name)); + } + + if (arrow_type->id() == arrow::Type::MAP) + { + const auto * arrow_map_type = typeid_cast(arrow_type.get()); + const auto * map_type = typeid_cast(column_type.get()); + if (!map_type) + throw Exception{"Cannot convert arrow MAP type to a not Map ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + + return std::make_shared( + getInternalType(arrow_map_type->key_type(), map_type->getKeyType(), column_name, format_name), + getInternalType(arrow_map_type->item_type(), map_type->getValueType(), column_name, format_name) + ); } if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 97a5326a421..7041193e09a 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -15,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -107,6 +109,7 @@ namespace DB size_t end, std::unordered_map> & dictionary_values); + template static void fillArrowArrayWithArrayColumnData( const String & column_name, ColumnPtr & column, @@ -118,12 +121,12 @@ namespace DB size_t end, std::unordered_map> & dictionary_values) { - const auto * column_array = typeid_cast(column.get()); + const auto * column_array = assert_cast(column.get()); ColumnPtr nested_column = column_array->getDataPtr(); - DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); + DataTypePtr nested_type = assert_cast(column_type.get())->getNestedType(); const auto & offsets = column_array->getOffsets(); - arrow::ListBuilder & builder = assert_cast(*array_builder); + Builder & builder = assert_cast(*array_builder); arrow::ArrayBuilder * value_builder = builder.value_builder(); arrow::Status components_status; @@ -147,8 +150,8 @@ namespace DB size_t end, std::unordered_map> & dictionary_values) { - const auto * column_tuple = typeid_cast(column.get()); - const auto & nested_types = typeid_cast(column_type.get())->getElements(); + const auto * column_tuple = assert_cast(column.get()); + const auto & nested_types = assert_cast(column_type.get())->getElements(); arrow::StructBuilder & builder = assert_cast(*array_builder); @@ -168,7 +171,7 @@ namespace DB template static PaddedPODArray extractIndexesImpl(ColumnPtr column, size_t start, size_t end) { - const PaddedPODArray & data = checkAndGetColumn>(*column)->getData(); + const PaddedPODArray & data = assert_cast *>(column.get())->getData(); PaddedPODArray result; result.reserve(end - start); std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value); }); @@ -205,21 +208,21 @@ namespace DB size_t end, std::unordered_map> & dictionary_values) { - const auto * column_lc = typeid_cast(column.get()); - arrow::DictionaryBuilder * builder = typeid_cast *>(array_builder); + const auto * column_lc = assert_cast(column.get()); + arrow::DictionaryBuilder * builder = assert_cast *>(array_builder); auto & dict_values = dictionary_values[column_name]; /// Convert dictionary from LowCardinality to Arrow dictionary only once and then reuse it. if (!dict_values) { - auto value_type = typeid_cast(builder->type().get())->value_type(); + const auto & value_type = assert_cast(builder->type().get())->value_type(); std::unique_ptr values_builder; arrow::MemoryPool* pool = arrow::default_memory_pool(); arrow::Status status = MakeBuilder(pool, value_type, &values_builder); checkStatus(status, column->getName(), format_name); auto dict_column = column_lc->getDictionary().getNestedColumn(); - const auto & dict_type = typeid_cast(column_type.get())->getDictionaryType(); + const auto & dict_type = assert_cast(column_type.get())->getDictionaryType(); fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, 0, dict_column->size(), dictionary_values); status = values_builder->Finish(&dict_values); checkStatus(status, column->getName(), format_name); @@ -259,8 +262,7 @@ namespace DB size_t end, std::unordered_map> & dictionary_values) { - auto * dict_type = typeid_cast(array_builder->type().get()); - auto value_type = dict_type->value_type(); + auto value_type = assert_cast(array_builder->type().get())->value_type(); #define DISPATCH(ARROW_TYPE_ID, ARROW_TYPE) \ if (arrow::Type::ARROW_TYPE_ID == value_type->id()) \ @@ -367,9 +369,9 @@ namespace DB if ("Nullable" == column_type_name) { - const ColumnNullable * column_nullable = checkAndGetColumn(column.get()); + const ColumnNullable * column_nullable = assert_cast(column.get()); ColumnPtr nested_column = column_nullable->getNestedColumnPtr(); - DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); + DataTypePtr nested_type = assert_cast(column_type.get())->getNestedType(); const ColumnPtr & null_column = column_nullable->getNullMapColumnPtr(); const PaddedPODArray & bytemap = assert_cast &>(*null_column).getData(); fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end, dictionary_values); @@ -392,7 +394,7 @@ namespace DB } else if ("Array" == column_type_name) { - fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); + fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); } else if ("Tuple" == column_type_name) { @@ -402,6 +404,12 @@ namespace DB { fillArrowArrayWithLowCardinalityColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); } + else if ("Map" == column_type_name) + { + ColumnPtr column_array = assert_cast(column.get())->getNestedColumnPtr(); + DataTypePtr array_type = assert_cast(column_type.get())->getNestedType(); + fillArrowArrayWithArrayColumnData(column_name, column_array, array_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); + } else if (isDecimal(column_type)) { auto fill_decimal = [&](const auto & types) -> bool @@ -446,7 +454,7 @@ namespace DB size_t start, size_t end) { - const auto & column = static_cast(*write_column); + const auto & column = assert_cast(*write_column); arrow::DecimalBuilder & builder = assert_cast(*array_builder); arrow::Status status; @@ -487,8 +495,8 @@ namespace DB { if (column_type->isNullable()) { - DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); - ColumnPtr nested_column = checkAndGetColumn(*column)->getNestedColumnPtr(); + DataTypePtr nested_type = assert_cast(column_type.get())->getNestedType(); + ColumnPtr nested_column = assert_cast(column.get())->getNestedColumnPtr(); auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable); *is_column_nullable = true; return arrow_type; @@ -506,7 +514,7 @@ namespace DB || std::is_same_v> || std::is_same_v>) { - const auto & decimal_type = static_cast(column_type.get()); + const auto & decimal_type = assert_cast(column_type.get()); arrow_type = arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale()); } @@ -518,16 +526,16 @@ namespace DB if (isArray(column_type)) { - auto nested_type = typeid_cast(column_type.get())->getNestedType(); - auto nested_column = checkAndGetColumn(*column)->getDataPtr(); + auto nested_type = assert_cast(column_type.get())->getNestedType(); + auto nested_column = assert_cast(column.get())->getDataPtr(); auto nested_arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable); return arrow::list(nested_arrow_type); } if (isTuple(column_type)) { - const auto & nested_types = typeid_cast(column_type.get())->getElements(); - const auto * tuple_column = checkAndGetColumn(*column); + const auto & nested_types = assert_cast(column_type.get())->getElements(); + const auto * tuple_column = assert_cast(column.get()); std::vector> nested_fields; for (size_t i = 0; i != nested_types.size(); ++i) { @@ -540,15 +548,28 @@ namespace DB if (column_type->lowCardinality()) { - auto nested_type = typeid_cast(column_type.get())->getDictionaryType(); - const auto * lc_column = checkAndGetColumn(*column); - auto nested_column = lc_column->getDictionaryPtr(); - auto indexes_column = lc_column->getIndexesPtr(); + auto nested_type = assert_cast(column_type.get())->getDictionaryType(); + const auto * lc_column = assert_cast(column.get()); + ColumnPtr nested_column = lc_column->getDictionaryPtr(); + ColumnPtr indexes_column = lc_column->getIndexesPtr(); return arrow::dictionary( getArrowTypeForLowCardinalityIndexes(indexes_column), getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable)); } + if (isMap(column_type)) + { + const auto * map_type = assert_cast(column_type.get()); + const auto & key_type = map_type->getKeyType(); + const auto & val_type = map_type->getValueType(); + + const auto & columns = assert_cast(column.get())->getNestedData().getColumns(); + return arrow::map( + getArrowType(key_type, columns[0], column_name, format_name, is_column_nullable), + getArrowType(val_type, columns[1], column_name, format_name, is_column_nullable) + ); + } + const std::string type_name = column_type->getFamilyName(); if (const auto * arrow_type_it = std::find_if( internal_type_to_arrow_type.begin(), @@ -559,7 +580,7 @@ namespace DB return arrow_type_it->second; } - throw Exception{"The type \"" + column_name + "\" of a column \"" + column_name + "\"" + throw Exception{"The type \"" + column_type->getName() + "\" of a column \"" + column_name + "\"" " is not supported for conversion into a " + format_name + " data format", ErrorCodes::UNKNOWN_TYPE}; } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 7b1c2af1a52..cb503def801 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -72,15 +72,25 @@ static size_t countIndicesForType(std::shared_ptr type) if (type->id() == arrow::Type::LIST) return countIndicesForType(static_cast(type.get())->value_type()) + 1; - int indices = 1; if (type->id() == arrow::Type::STRUCT) { + int indices = 1; auto * struct_type = static_cast(type.get()); for (int i = 0; i != struct_type->num_fields(); ++i) indices += countIndicesForType(struct_type->field(i)->type()); + return indices; } - return indices; + if (type->id() == arrow::Type::MAP) + { + int indices = 0; + auto * map_type = static_cast(type.get()); + indices += countIndicesForType(map_type->key_type()); + indices += countIndicesForType(map_type->item_type()); + return indices; + } + + return 1; } void ORCBlockInputFormat::prepareReader() diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index a560701944d..a5143792e7d 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -18,6 +19,7 @@ #include #include #include +#include namespace DB { @@ -102,27 +104,27 @@ ORC_UNIQUE_PTR ORCBlockOutputFormat::getORCType(const DataTypePtr & t } case TypeIndex::Array: { - const auto * array_type = typeid_cast(type.get()); + const auto * array_type = assert_cast(type.get()); return orc::createListType(getORCType(array_type->getNestedType(), column_name)); } case TypeIndex::Decimal32: { - const auto * decimal_type = typeid_cast *>(type.get()); + const auto * decimal_type = assert_cast *>(type.get()); return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale()); } case TypeIndex::Decimal64: { - const auto * decimal_type = typeid_cast *>(type.get()); + const auto * decimal_type = assert_cast *>(type.get()); return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale()); } case TypeIndex::Decimal128: { - const auto * decimal_type = typeid_cast *>(type.get()); + const auto * decimal_type = assert_cast *>(type.get()); return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale()); } case TypeIndex::Tuple: { - const auto * tuple_type = typeid_cast(type.get()); + const auto * tuple_type = assert_cast(type.get()); const auto & nested_types = tuple_type->getElements(); auto struct_type = orc::createStructType(); for (size_t i = 0; i < nested_types.size(); ++i) @@ -132,6 +134,14 @@ ORC_UNIQUE_PTR ORCBlockOutputFormat::getORCType(const DataTypePtr & t } return struct_type; } + case TypeIndex::Map: + { + const auto * map_type = assert_cast(type.get()); + return orc::createMapType( + getORCType(map_type->getKeyType(), column_name), + getORCType(map_type->getValueType(), column_name) + ); + } default: { throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN); @@ -174,7 +184,7 @@ void ORCBlockOutputFormat::writeDecimals( { DecimalVectorBatch & decimal_orc_column = dynamic_cast(orc_column); const auto & decimal_column = assert_cast &>(column); - const auto * decimal_type = typeid_cast *>(type.get()); + const auto * decimal_type = assert_cast *>(type.get()); decimal_orc_column.precision = decimal_type->getPrecision(); decimal_orc_column.scale = decimal_type->getScale(); decimal_orc_column.resize(decimal_column.size()); @@ -400,13 +410,40 @@ void ORCBlockOutputFormat::writeColumn( { orc::StructVectorBatch & struct_orc_column = dynamic_cast(orc_column); const auto & tuple_column = assert_cast(column); - auto nested_types = typeid_cast(type.get())->getElements(); + auto nested_types = assert_cast(type.get())->getElements(); for (size_t i = 0; i != tuple_column.size(); ++i) struct_orc_column.notNull[i] = 1; for (size_t i = 0; i != tuple_column.tupleSize(); ++i) writeColumn(*struct_orc_column.fields[i], tuple_column.getColumn(i), nested_types[i], null_bytemap); break; + } + case TypeIndex::Map: + { + orc::MapVectorBatch & map_orc_column = dynamic_cast(orc_column); + const auto & list_column = assert_cast(column).getNestedColumn(); + const auto & map_type = assert_cast(*type); + const ColumnArray::Offsets & offsets = list_column.getOffsets(); + map_orc_column.resize(list_column.size()); + /// The length of list i in ListVectorBatch is offsets[i+1] - offsets[i]. + map_orc_column.offsets[0] = 0; + for (size_t i = 0; i != list_column.size(); ++i) + { + map_orc_column.offsets[i + 1] = offsets[i]; + map_orc_column.notNull[i] = 1; + } + const auto nested_columns = assert_cast(list_column.getDataPtr().get())->getColumns(); + + orc::ColumnVectorBatch & keys_orc_column = *map_orc_column.keys; + auto key_type = map_type.getKeyType(); + writeColumn(keys_orc_column, *nested_columns[0], key_type, null_bytemap); + + orc::ColumnVectorBatch & values_orc_column = *map_orc_column.elements; + auto value_type = map_type.getValueType(); + writeColumn(values_orc_column, *nested_columns[1], value_type, null_bytemap); + + map_orc_column.numElements = list_column.size(); + break; } default: throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN); diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 1d7feadc1e7..bde39446dbb 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -79,6 +79,12 @@ static size_t countIndicesForType(std::shared_ptr type) for (int i = 0; i != struct_type->num_fields(); ++i) indices += countIndicesForType(struct_type->field(i)->type()); } + else if (type->id() == arrow::Type::MAP) + { + auto * map_type = static_cast(type.get()); + indices += countIndicesForType(map_type->key_type()); + indices += countIndicesForType(map_type->item_type()); + } else indices = 1; diff --git a/tests/queries/0_stateless/00900_long_parquet_load.reference b/tests/queries/0_stateless/00900_long_parquet_load.reference index 4bd699f40fe..621bca2ec0e 100644 --- a/tests/queries/0_stateless/00900_long_parquet_load.reference +++ b/tests/queries/0_stateless/00900_long_parquet_load.reference @@ -298,24 +298,40 @@ Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not y [[['a','b'],['c','d']],[[],['e']]] 1 [[['a','b'],['c','d'],['e']],[[],['f']]] 1 === Try load data from nested_maps.snappy.parquet -Code: 70. DB::Ex---tion: The type "map" of an input column "a" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin - +{'a':{1:1,2:0}} 1 1 +{'b':{1:1}} 1 1 +{'c':{}} 1 1 +{'d':{}} 1 1 +{'e':{1:1}} 1 1 +{'f':{3:1,4:0,5:1}} 1 1 === Try load data from non_hadoop_lz4_compressed.parquet 1593604800 abc 42 1593604800 def 7.7 1593604801 abc 42.125 1593604801 def 7.7 === Try load data from nonnullable.impala.parquet -../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id()) +8 [-1] [[-1,-2],[]] {'k1':-1} [{},{'k1':1},{},{}] (-1,[-1],([[(-1)]]),{}) === Try load data from nullable.impala.parquet -../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id()) +1 [1,2,3] [[1,2],[3,4]] {'k1':1,'k2':100} [{'k1':1}] (1,[1],([[(10),(-10)],[(11)]]),{'foo':(([1.1]))}) +2 [NULL,1,2,NULL,3,NULL] [[NULL,1,2,NULL],[3,NULL,4],[],[]] {'k1':2,'k2':NULL} [{'k3':NULL,'k1':1},{},{}] (NULL,[NULL],([[(NULL),(10),(NULL),(-10),(NULL)],[(11),(NULL)],[],[]]),{'g1':(([2.2,NULL])),'g2':(([])),'g3':(([])),'g4':(([])),'g5':(([]))}) +3 [] [[]] {} [{},{}] (NULL,[],([]),{}) +4 [] [] {} [] (NULL,[],([]),{}) +5 [] [] {} [] (NULL,[],([]),{'foo':(([2.2,3.3]))}) +6 [] [] {} [] (NULL,[],([]),{}) +7 [] [[],[5,6]] {'k1':NULL,'k3':NULL} [] (7,[2,3,NULL],([[],[(NULL)],[]]),{}) === Try load data from nullable_list.parquet [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] [NULL] [NULL] [NULL] [] [] [] === Try load data from nulls.snappy.parquet -Code: 70. DB::Ex---tion: The type "struct" of an input column "b_struct" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin - +(NULL) +(NULL) +(NULL) +(NULL) +(NULL) +(NULL) +(NULL) +(NULL) === Try load data from single_nan.parquet \N === Try load data from userdata1.parquet diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 52213f066e1..1cfba22587a 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -55,7 +55,10 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A COLUMNS=$(cat "$COLUMNS_FILE") || continue ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load" - ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory" + $CLICKHOUSE_CLIENT --multiquery <&1 | sed 's/Exception/Ex---tion/' diff --git a/tests/queries/0_stateless/00900_orc_arrow_parquet_maps.reference b/tests/queries/0_stateless/00900_orc_arrow_parquet_maps.reference new file mode 100644 index 00000000000..d96eb672258 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_arrow_parquet_maps.reference @@ -0,0 +1,6 @@ +Arrow +{1:2,2:3} {'1':'a','2':'b'} {1:(1,2),2:(3,4)} {1:[1,2],2:[3,4]} [{1:2,2:3},{3:4,4:5}] ({1:2,2:3},{'a':'b','c':'d'}) [{1:[({1:2},(1)),({2:3},(2))]},{2:[({3:4},(3)),({4:5},(4))]}] +Parquet +{1:2,2:3} {'1':'a','2':'b'} {1:(1,2),2:(3,4)} {1:[1,2],2:[3,4]} [{1:2,2:3},{3:4,4:5}] ({1:2,2:3},{'a':'b','c':'d'}) [{1:[({1:2},(1)),({2:3},(2))]},{2:[({3:4},(3)),({4:5},(4))]}] +ORC +{1:2,2:3} {'1':'a','2':'b'} {1:(1,2),2:(3,4)} {1:[1,2],2:[3,4]} [{1:2,2:3},{3:4,4:5}] ({1:2,2:3},{'a':'b','c':'d'}) [{1:[({1:2},(1)),({2:3},(2))]},{2:[({3:4},(3)),({4:5},(4))]}] diff --git a/tests/queries/0_stateless/00900_orc_arrow_parquet_maps.sh b/tests/queries/0_stateless/00900_orc_arrow_parquet_maps.sh new file mode 100755 index 00000000000..9330a5924a9 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_arrow_parquet_maps.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS maps" +${CLICKHOUSE_CLIENT} --multiquery < "${CLICKHOUSE_TMP}"/maps + ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE maps" + cat "${CLICKHOUSE_TMP}"/maps | ${CLICKHOUSE_CLIENT} -q "INSERT INTO maps FORMAT Parquet" + ${CLICKHOUSE_CLIENT} --query="SELECT * FROM maps" +done + +${CLICKHOUSE_CLIENT} --query="DROP TABLE maps" diff --git a/tests/queries/0_stateless/00900_orc_tuples_load.reference b/tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.reference similarity index 63% rename from tests/queries/0_stateless/00900_orc_tuples_load.reference rename to tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.reference index cbf8ee819f7..12556ca2c7b 100644 --- a/tests/queries/0_stateless/00900_orc_tuples_load.reference +++ b/tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.reference @@ -1,2 +1,6 @@ +Arrow (1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] +Parquet +(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] +ORC (1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] diff --git a/tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.sh b/tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.sh new file mode 100755 index 00000000000..ff16d3205ad --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_arrow_parquet_tuples.sh @@ -0,0 +1,24 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS tuples"; +${CLICKHOUSE_CLIENT} --query="CREATE TABLE tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])" + +formats="Arrow Parquet ORC"; + +for format in ${formats}; do + echo $format + + ${CLICKHOUSE_CLIENT} --query="SELECT * FROM tuples FORMAT $format" > "${CLICKHOUSE_TMP}"/tuples + ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE tuples" + cat "${CLICKHOUSE_TMP}"/tuples | ${CLICKHOUSE_CLIENT} -q "INSERT INTO tuples FORMAT $format" + ${CLICKHOUSE_CLIENT} --query="SELECT * FROM tuples" +done + +${CLICKHOUSE_CLIENT} --query="DROP TABLE tuples" diff --git a/tests/queries/0_stateless/00900_orc_map_load.sh b/tests/queries/0_stateless/00900_orc_map_load.sh new file mode 100644 index 00000000000..3ddef17bd5c --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_map_load.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_maps" +${CLICKHOUSE_CLIENT} --multiquery < "${CLICKHOUSE_TMP}"/maps.parquet + +cat "${CLICKHOUSE_TMP}"/maps.parquet | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_maps FORMAT Parquet" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_maps" +${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_maps" diff --git a/tests/queries/0_stateless/00900_orc_tuples_load.sh b/tests/queries/0_stateless/00900_orc_tuples_load.sh deleted file mode 100755 index f32be6af751..00000000000 --- a/tests/queries/0_stateless/00900_orc_tuples_load.sh +++ /dev/null @@ -1,17 +0,0 @@ -#!/usr/bin/env bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_tuples" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()" - -${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])" - -${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_tuples FORMAT ORC" > "${CLICKHOUSE_TMP}"/tuples.orc - -cat "${CLICKHOUSE_TMP}"/tuples.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_tuples FORMAT ORC" - -${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_tuples" -${CLICKHOUSE_CLIENT} --query="DROP TABLE orc_tuples" diff --git a/tests/queries/0_stateless/00900_parquet_tuples_load.reference b/tests/queries/0_stateless/00900_parquet_tuples_load.reference deleted file mode 100644 index cbf8ee819f7..00000000000 --- a/tests/queries/0_stateless/00900_parquet_tuples_load.reference +++ /dev/null @@ -1,2 +0,0 @@ -(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] -(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] diff --git a/tests/queries/0_stateless/00900_parquet_tuples_load.sh b/tests/queries/0_stateless/00900_parquet_tuples_load.sh deleted file mode 100755 index 031e8111019..00000000000 --- a/tests/queries/0_stateless/00900_parquet_tuples_load.sh +++ /dev/null @@ -1,17 +0,0 @@ -#!/usr/bin/env bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_tuples" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()" - -${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])" - -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_tuples FORMAT Parquet" > "${CLICKHOUSE_TMP}"/tuples.parquet - -cat "${CLICKHOUSE_TMP}"/tuples.parquet | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_tuples FORMAT Parquet" - -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_tuples" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_tuples" diff --git a/tests/queries/0_stateless/01273_arrow_tuples_load.reference b/tests/queries/0_stateless/01273_arrow_tuples_load.reference deleted file mode 100644 index cbf8ee819f7..00000000000 --- a/tests/queries/0_stateless/01273_arrow_tuples_load.reference +++ /dev/null @@ -1,2 +0,0 @@ -(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] -(1,2) ('1','2') ((1,'1'),1) ((1,2),('1','2')) ([1,2,3],1) (([1,2,3],[1,2,3]),([[1,2,3],[1,2,3]],1)) [([[1,2,3],[1,2,3]],([(1,2),(1,2)],1))] diff --git a/tests/queries/0_stateless/01273_arrow_tuples_load.sh b/tests/queries/0_stateless/01273_arrow_tuples_load.sh deleted file mode 100755 index 311079afe50..00000000000 --- a/tests/queries/0_stateless/01273_arrow_tuples_load.sh +++ /dev/null @@ -1,17 +0,0 @@ -#!/usr/bin/env bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_tuples" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_tuples (t1 Tuple(UInt32, UInt32), t2 Tuple(String, String), t3 Tuple(Tuple(UInt32, String), UInt32), t4 Tuple(Tuple(UInt32, UInt32), Tuple(String, String)), t5 Tuple(Array(UInt32), UInt32), t6 Tuple(Tuple(Array(UInt32), Array(UInt32)), Tuple(Array(Array(UInt32)), UInt32)), t7 Array(Tuple(Array(Array(UInt32)), Tuple(Array(Tuple(UInt32, UInt32)), UInt32)))) ENGINE=Memory()" - -${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_tuples VALUES ((1, 2), ('1', '2'), ((1, '1'), 1), ((1, 2), ('1', '2')), ([1,2,3], 1), (([1,2,3], [1,2,3]), ([[1,2,3], [1,2,3]], 1)), [([[1,2,3], [1,2,3]], ([(1, 2), (1, 2)], 1))])" - -${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_tuples FORMAT Arrow" > "${CLICKHOUSE_TMP}"/tuples.arrow - -cat "${CLICKHOUSE_TMP}"/tuples.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_tuples FORMAT Arrow" - -${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_tuples" -${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_tuples" diff --git a/tests/queries/0_stateless/data_parquet/nested_maps.snappy.parquet.columns b/tests/queries/0_stateless/data_parquet/nested_maps.snappy.parquet.columns index d5e9599431b..eef66ae66c7 100644 --- a/tests/queries/0_stateless/data_parquet/nested_maps.snappy.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/nested_maps.snappy.parquet.columns @@ -1 +1 @@ -`a` Tuple(Nullable(String), Nullable(Int32), Nullable(UInt8)), `b` Nullable(Int32), `c` Nullable(Float64) +`a` Map(String, Map(Int32, Nullable(UInt8))), `b` Nullable(Int32), `c` Nullable(Float64) diff --git a/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns b/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns index 6d724200aec..299ec3b6af2 100644 --- a/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/nonnullable.impala.parquet.columns @@ -1 +1 @@ -`ID` Nullable(Int64), `Int_Array` Nullable(Int32), `int_array_array` Nullable(Int32), `Int_Map` Tuple(Nullable(String), Nullable(Int32)), `int_map_array` Tuple(Nullable(String), Nullable(Int32)), `nested_Struct` Tuple(Nullable(Int32), Nullable(Int32), Nullable(Int32), Nullable(String), Nullable(String), Nullable(Float64)) +`ID` Nullable(Int64), `Int_Array` Array(Nullable(Int32)), `int_array_array` Array(Array(Nullable(Int32))), `Int_Map` Map(String, Nullable(Int32)), `int_map_array` Array(Map(String, Nullable(Int32))), `nested_Struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32))))), Map(String, Tuple(Tuple(Array(Nullable(Float64)))))) diff --git a/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns b/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns index b5e122585d7..6fcbcdd4a0b 100644 --- a/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/nullable.impala.parquet.columns @@ -1 +1 @@ -`id` Nullable(Int64), `int_array` Nullable(Int32), `int_array_Array` Nullable(Int32), `int_map` Tuple(Nullable(String), Nullable(Int32)), `int_Map_Array` Tuple(Nullable(String), Nullable(Int32)), `nested_struct` Tuple(Nullable(Int32), Nullable(Int32), Nullable(Int32), Nullable(String), Nullable(String), Nullable(Float64)) +`id` Nullable(Int64), `int_array` Array(Nullable(Int32)), `int_array_Array` Array(Array(Nullable(Int32))), `int_map` Map(String, Nullable(Int32)), `int_Map_Array` Array(Map(String, Nullable(Int32))), `nested_struct` Tuple(Nullable(Int32), Array(Nullable(Int32)), Tuple(Array(Array(Tuple(Nullable(Int32))))), Map(String, Tuple(Tuple(Array(Nullable(Float64)))))) diff --git a/tests/queries/0_stateless/data_parquet/nulls.snappy.parquet.columns b/tests/queries/0_stateless/data_parquet/nulls.snappy.parquet.columns index a99b8b80eac..6e723ef72c3 100644 --- a/tests/queries/0_stateless/data_parquet/nulls.snappy.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/nulls.snappy.parquet.columns @@ -1 +1 @@ -`b_struct` Nullable(Int32) +`b_struct` Tuple(Nullable(Int32)) From b120841b579662f97734667d2c59e98382b8c669 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 27 May 2021 22:10:45 +0300 Subject: [PATCH 07/43] Small changes --- src/DataTypes/DataTypeLowCardinality.h | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 2 +- src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 5 +---- .../Formats/Impl/ParquetBlockInputFormat.cpp | 13 ++++++------- 4 files changed, 9 insertions(+), 13 deletions(-) diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 2ab62fabf41..1266174c6d6 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -13,7 +13,7 @@ private: DataTypePtr dictionary_type; public: - DataTypeLowCardinality(DataTypePtr dictionary_type_dict); + DataTypeLowCardinality(DataTypePtr dictionary_type_); const DataTypePtr & getDictionaryType() const { return dictionary_type; } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 8f76f5dca8a..755cc03fa84 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -454,7 +454,7 @@ namespace DB const DataTypeArray * array_type = typeid_cast(column_type.get()); if (!array_type) - throw Exception{"Cannot convert arrow LIST type to a not Array/Map ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + throw Exception{"Cannot convert arrow LIST type to a not Array ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; return std::make_shared(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name)); } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index cb503def801..0f8ca728c46 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -83,11 +83,8 @@ static size_t countIndicesForType(std::shared_ptr type) if (type->id() == arrow::Type::MAP) { - int indices = 0; auto * map_type = static_cast(type.get()); - indices += countIndicesForType(map_type->key_type()); - indices += countIndicesForType(map_type->item_type()); - return indices; + return countIndicesForType(map_type->key_type()) + countIndicesForType(map_type->item_type()); } return 1; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index bde39446dbb..ea7d35f5bbe 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -72,23 +72,22 @@ static size_t countIndicesForType(std::shared_ptr type) if (type->id() == arrow::Type::LIST) return countIndicesForType(static_cast(type.get())->value_type()); - int indices = 0; if (type->id() == arrow::Type::STRUCT) { + int indices = 0; auto * struct_type = static_cast(type.get()); for (int i = 0; i != struct_type->num_fields(); ++i) indices += countIndicesForType(struct_type->field(i)->type()); + return indices; } - else if (type->id() == arrow::Type::MAP) + + if (type->id() == arrow::Type::MAP) { auto * map_type = static_cast(type.get()); - indices += countIndicesForType(map_type->key_type()); - indices += countIndicesForType(map_type->item_type()); + return countIndicesForType(map_type->key_type()) + countIndicesForType(map_type->item_type()); } - else - indices = 1; - return indices; + return 1; } void ParquetBlockInputFormat::prepareReader() From 787c8000641a44e0bfe35de471ade5d7160c780f Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 1 Jun 2021 10:37:05 +0300 Subject: [PATCH 08/43] Fix build, tests, style --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 20 ++++++++++--------- .../Formats/Impl/ArrowColumnToCHColumn.h | 4 ++-- .../Formats/Impl/CHColumnToArrowColumn.cpp | 7 ++++--- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 1 - 4 files changed, 17 insertions(+), 15 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 755cc03fa84..4fa5e190b6c 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -33,6 +33,7 @@ namespace DB extern const int CANNOT_CONVERT_TYPE; extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int THERE_IS_NO_COLUMN; + extern const int BAD_ARGUMENTS; } static const std::initializer_list> arrow_type_to_internal_type = @@ -269,19 +270,21 @@ namespace DB offsets_data.emplace_back(start + arrow_offsets.Value(i)); } } - - static DataTypePtr getInternalIndexesType(std::shared_ptr arrow_type) + static ColumnPtr createAndFillColumnWithIndexesData(std::shared_ptr & arrow_column) { - switch (arrow_type->id()) + switch (arrow_column->type()->id()) { # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ - return std::make_shared>(); \ - - FOR_ARROW_INXEXES_TYPES(DISPATCH) + { \ + auto column = DataTypeNumber().createColumn(); \ + fillColumnWithNumericData(arrow_column, *column); \ + return column; \ + } + FOR_ARROW_INDEXES_TYPES(DISPATCH) # undef DISPATCH default: - throw Exception("Unsupported type for indexes in LowCardinality: " + arrow_type->name() + ".", ErrorCodes::BAD_ARGUMENTS); + throw Exception("Unsupported type for indexes in LowCardinality: " + arrow_column->type()->name() + ".", ErrorCodes::BAD_ARGUMENTS); } } @@ -406,9 +409,8 @@ namespace DB } auto arrow_indexes_column = std::make_shared(indexes_array); - auto indexes_column = getInternalIndexesType(arrow_indexes_column->type())->createColumn(); + auto indexes_column = createAndFillColumnWithIndexesData(arrow_indexes_column); - readColumnFromArrowColumn(arrow_indexes_column, *indexes_column, column_name, format_name, is_nullable, dictionary_values); auto new_column_lc = ColumnLowCardinality::create(dict_values, std::move(indexes_column)); column_lc = std::move(*new_column_lc); break; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 60fbc30e9c9..0ba2d891a2c 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -36,7 +36,7 @@ namespace DB M(arrow::Type::FLOAT, DB::Float32) \ M(arrow::Type::DOUBLE, DB::Float64) -# define FOR_ARROW_INXEXES_TYPES(M) \ +# define FOR_ARROW_INDEXES_TYPES(M) \ M(arrow::Type::UINT8, DB::UInt8) \ M(arrow::Type::INT8, DB::UInt8) \ M(arrow::Type::UINT16, DB::UInt16) \ @@ -44,7 +44,7 @@ namespace DB M(arrow::Type::UINT32, DB::UInt32) \ M(arrow::Type::INT32, DB::UInt32) \ M(arrow::Type::UINT64, DB::UInt64) \ - M(arrow::Type::INT64, DB::UInt64) \ + M(arrow::Type::INT64, DB::UInt64) /// Map {column name : dictionary column}. /// To avoid converting dictionary from Arrow Dictionary diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 7041193e09a..c0909ac3d22 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -30,6 +30,7 @@ namespace DB { extern const int UNKNOWN_EXCEPTION; extern const int UNKNOWN_TYPE; + extern const int LOGICAL_ERROR; } static const std::initializer_list>> internal_type_to_arrow_type = @@ -215,7 +216,7 @@ namespace DB /// Convert dictionary from LowCardinality to Arrow dictionary only once and then reuse it. if (!dict_values) { - const auto & value_type = assert_cast(builder->type().get())->value_type(); + auto value_type = assert_cast(builder->type().get())->value_type(); std::unique_ptr values_builder; arrow::MemoryPool* pool = arrow::default_memory_pool(); arrow::Status status = MakeBuilder(pool, value_type, &values_builder); @@ -550,8 +551,8 @@ namespace DB { auto nested_type = assert_cast(column_type.get())->getDictionaryType(); const auto * lc_column = assert_cast(column.get()); - ColumnPtr nested_column = lc_column->getDictionaryPtr(); - ColumnPtr indexes_column = lc_column->getIndexesPtr(); + const auto & nested_column = lc_column->getDictionaryPtr(); + const auto & indexes_column = lc_column->getIndexesPtr(); return arrow::dictionary( getArrowTypeForLowCardinalityIndexes(indexes_column), getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable)); diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 25eade4bafa..1eab69239ca 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include "ArrowBufferedStreams.h" #include "CHColumnToArrowColumn.h" From 02e68655b423488b3d02ea63b4ba37b92c4096ec Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 1 Jun 2021 11:32:09 +0300 Subject: [PATCH 09/43] Update orc and arrow --- .gitmodules | 2 +- contrib/arrow | 2 +- contrib/arrow-cmake/CMakeLists.txt | 37 ++++++++++++++++++++++-------- contrib/orc | 2 +- 4 files changed, 30 insertions(+), 13 deletions(-) diff --git a/.gitmodules b/.gitmodules index ab7c8a7c94d..2e9ea26dcac 100644 --- a/.gitmodules +++ b/.gitmodules @@ -103,7 +103,7 @@ url = https://github.com/ClickHouse-Extras/fastops [submodule "contrib/orc"] path = contrib/orc - url = https://github.com/apache/orc + url = https://github.com/ClickHouse-Extras/orc [submodule "contrib/sparsehash-c11"] path = contrib/sparsehash-c11 url = https://github.com/sparsehash/sparsehash-c11.git diff --git a/contrib/arrow b/contrib/arrow index 616b3dc76a0..debf751a129 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 616b3dc76a0c8450b4027ded8a78e9619d7c845f +Subproject commit debf751a129bdda9ff4d1e895e08957ff77000a1 diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index deefb244beb..069b96dd006 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -188,6 +188,7 @@ set(ARROW_SRCS "${LIBRARY_DIR}/array/util.cc" "${LIBRARY_DIR}/array/validate.cc" + "${LIBRARY_DIR}/compute/api_aggregate.cc" "${LIBRARY_DIR}/compute/api_scalar.cc" "${LIBRARY_DIR}/compute/api_vector.cc" "${LIBRARY_DIR}/compute/cast.cc" @@ -198,8 +199,11 @@ set(ARROW_SRCS "${LIBRARY_DIR}/compute/kernels/aggregate_basic.cc" "${LIBRARY_DIR}/compute/kernels/aggregate_mode.cc" + "${LIBRARY_DIR}/compute/kernels/aggregate_quantile.cc" + "${LIBRARY_DIR}/compute/kernels/aggregate_tdigest.cc" "${LIBRARY_DIR}/compute/kernels/aggregate_var_std.cc" "${LIBRARY_DIR}/compute/kernels/codegen_internal.cc" + "${LIBRARY_DIR}/compute/kernels/hash_aggregate.cc" "${LIBRARY_DIR}/compute/kernels/scalar_arithmetic.cc" "${LIBRARY_DIR}/compute/kernels/scalar_boolean.cc" "${LIBRARY_DIR}/compute/kernels/scalar_cast_boolean.cc" @@ -243,6 +247,7 @@ set(ARROW_SRCS "${LIBRARY_DIR}/io/interfaces.cc" "${LIBRARY_DIR}/io/memory.cc" "${LIBRARY_DIR}/io/slow.cc" + "${LIBRARY_DIR}/io/transform.cc" "${LIBRARY_DIR}/tensor/coo_converter.cc" "${LIBRARY_DIR}/tensor/csf_converter.cc" @@ -256,11 +261,8 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/bitmap_builders.cc" "${LIBRARY_DIR}/util/bitmap_ops.cc" "${LIBRARY_DIR}/util/bpacking.cc" + "${LIBRARY_DIR}/util/cancel.cc" "${LIBRARY_DIR}/util/compression.cc" - "${LIBRARY_DIR}/util/compression_lz4.cc" - "${LIBRARY_DIR}/util/compression_snappy.cc" - "${LIBRARY_DIR}/util/compression_zlib.cc" - "${LIBRARY_DIR}/util/compression_zstd.cc" "${LIBRARY_DIR}/util/cpu_info.cc" "${LIBRARY_DIR}/util/decimal.cc" "${LIBRARY_DIR}/util/delimiting.cc" @@ -268,20 +270,33 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/future.cc" "${LIBRARY_DIR}/util/int_util.cc" "${LIBRARY_DIR}/util/io_util.cc" - "${LIBRARY_DIR}/util/iterator.cc" "${LIBRARY_DIR}/util/key_value_metadata.cc" "${LIBRARY_DIR}/util/logging.cc" "${LIBRARY_DIR}/util/memory.cc" + "${LIBRARY_DIR}/util/mutex.cc" "${LIBRARY_DIR}/util/string_builder.cc" "${LIBRARY_DIR}/util/string.cc" "${LIBRARY_DIR}/util/task_group.cc" + "${LIBRARY_DIR}/util/tdigest.cc" "${LIBRARY_DIR}/util/thread_pool.cc" "${LIBRARY_DIR}/util/time.cc" "${LIBRARY_DIR}/util/trie.cc" + "${LIBRARY_DIR}/util/uri.cc" "${LIBRARY_DIR}/util/utf8.cc" "${LIBRARY_DIR}/util/value_parsing.cc" "${LIBRARY_DIR}/vendored/base64.cpp" + "${LIBRARY_DIR}/vendored/datetime/tz.cpp" + "${LIBRARY_DIR}/vendored/double-conversion/bignum.cc" + "${LIBRARY_DIR}/vendored/double-conversion/double-conversion.cc" + "${LIBRARY_DIR}/vendored/double-conversion/bignum-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/fast-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/cached-powers.cc" + "${LIBRARY_DIR}/vendored/double-conversion/fixed-dtoa.cc" + "${LIBRARY_DIR}/vendored/double-conversion/diy-fp.cc" + "${LIBRARY_DIR}/vendored/double-conversion/strtod.cc" + + "${LIBRARY_DIR}/c/bridge.cc" ${ORC_SRCS} ) @@ -368,14 +383,14 @@ set(PARQUET_SRCS "${LIBRARY_DIR}/column_reader.cc" "${LIBRARY_DIR}/column_scanner.cc" "${LIBRARY_DIR}/column_writer.cc" - "${LIBRARY_DIR}/deprecated_io.cc" "${LIBRARY_DIR}/encoding.cc" - "${LIBRARY_DIR}/encryption.cc" - "${LIBRARY_DIR}/encryption_internal.cc" + "${LIBRARY_DIR}/encryption/encryption.cc" + "${LIBRARY_DIR}/encryption/encryption_internal.cc" + "${LIBRARY_DIR}/encryption/internal_file_decryptor.cc" + "${LIBRARY_DIR}/encryption/internal_file_encryptor.cc" + "${LIBRARY_DIR}/exception.cc" "${LIBRARY_DIR}/file_reader.cc" "${LIBRARY_DIR}/file_writer.cc" - "${LIBRARY_DIR}/internal_file_decryptor.cc" - "${LIBRARY_DIR}/internal_file_encryptor.cc" "${LIBRARY_DIR}/level_conversion.cc" "${LIBRARY_DIR}/level_comparison.cc" "${LIBRARY_DIR}/metadata.cc" @@ -385,6 +400,8 @@ set(PARQUET_SRCS "${LIBRARY_DIR}/properties.cc" "${LIBRARY_DIR}/schema.cc" "${LIBRARY_DIR}/statistics.cc" + "${LIBRARY_DIR}/stream_reader.cc" + "${LIBRARY_DIR}/stream_writer.cc" "${LIBRARY_DIR}/types.cc" "${GEN_LIBRARY_DIR}/parquet_constants.cpp" diff --git a/contrib/orc b/contrib/orc index 5981208e394..0a936f6bbdb 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit 5981208e39447df84827f6a961d1da76bacb6078 +Subproject commit 0a936f6bbdb9303308973073f8623b5a8d82eae1 From eb86721d14fae27786058740dd55282440839d3c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 1 Jun 2021 19:42:05 +0300 Subject: [PATCH 10/43] Fix style, output strings as binary instead of utf8 --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 8 +++++--- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 7 +++---- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 4fa5e190b6c..2dad3a15026 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -472,10 +472,12 @@ namespace DB int internal_fields_num = tuple_nested_types.size(); /// If internal column has less elements then arrow struct, we will select only first internal_fields_num columns. if (internal_fields_num > struct_type->num_fields()) - throw Exception{ - "Cannot convert arrow STRUCT with " + std::to_string(struct_type->num_fields()) + " fields to a ClickHouse Tuple with " + throw Exception + { + "Cannot convert arrow STRUCT with " + std::to_string(struct_type->num_fields()) + " fields to a ClickHouse Tuple with " + std::to_string(internal_fields_num) + " elements " + column_type->getName(), - ErrorCodes::CANNOT_CONVERT_TYPE}; + ErrorCodes::CANNOT_CONVERT_TYPE + }; DataTypes nested_types; for (int i = 0; i < internal_fields_num; ++i) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index c0909ac3d22..c99c80c8d84 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -52,9 +52,8 @@ namespace DB //{"DateTime", arrow::date64()}, // BUG! saves as date32 {"DateTime", arrow::uint32()}, - // TODO: ClickHouse can actually store non-utf8 strings! - {"String", arrow::utf8()}, - {"FixedString", arrow::utf8()}, + {"String", arrow::binary()}, + {"FixedString", arrow::binary()}, }; @@ -287,7 +286,7 @@ namespace DB size_t end) { const auto & internal_column = assert_cast(*write_column); - arrow::StringBuilder & builder = assert_cast(*array_builder); + arrow::BinaryBuilder & builder = assert_cast(*array_builder); arrow::Status status; for (size_t string_i = start; string_i < end; ++string_i) From 83c843a20bb667bd32030cedf8e23b11cdcf3939 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 2 Jun 2021 00:47:18 +0300 Subject: [PATCH 11/43] Fix tests and build --- contrib/arrow-cmake/CMakeLists.txt | 1 - .../queries/0_stateless/00900_orc_map_load.sh | 20 ------------------- .../01273_arrow_dictionaries_load.reference | 2 -- 3 files changed, 23 deletions(-) delete mode 100644 tests/queries/0_stateless/00900_orc_map_load.sh diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 069b96dd006..878b46b39a6 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -281,7 +281,6 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/thread_pool.cc" "${LIBRARY_DIR}/util/time.cc" "${LIBRARY_DIR}/util/trie.cc" - "${LIBRARY_DIR}/util/uri.cc" "${LIBRARY_DIR}/util/utf8.cc" "${LIBRARY_DIR}/util/value_parsing.cc" diff --git a/tests/queries/0_stateless/00900_orc_map_load.sh b/tests/queries/0_stateless/00900_orc_map_load.sh deleted file mode 100644 index 3ddef17bd5c..00000000000 --- a/tests/queries/0_stateless/00900_orc_map_load.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/usr/bin/env bash - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_maps" -${CLICKHOUSE_CLIENT} --multiquery < "${CLICKHOUSE_TMP}"/maps.parquet - -cat "${CLICKHOUSE_TMP}"/maps.parquet | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_maps FORMAT Parquet" - -${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_maps" -${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_maps" diff --git a/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference b/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference index b144130918f..7321c396a59 100644 --- a/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference +++ b/tests/queries/0_stateless/01273_arrow_dictionaries_load.reference @@ -1,4 +1,2 @@ 1 ['a','b','c'] ('z','6') 2 ['d','e'] ('x','9') -1 ['a','b','c'] ('z','6') -2 ['d','e'] ('x','9') From a4ef60e230d9244a49831486f9bdcf24281d02da Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 2 Jun 2021 11:51:07 +0300 Subject: [PATCH 12/43] Remove Impl including from .h file --- src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp | 5 +++-- src/Processors/Formats/Impl/ArrowBlockInputFormat.h | 4 ++-- src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp | 8 ++++++-- src/Processors/Formats/Impl/ArrowBlockOutputFormat.h | 5 +++-- src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/ORCBlockInputFormat.h | 6 ++++-- src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/ParquetBlockInputFormat.h | 5 +++-- src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/ParquetBlockOutputFormat.h | 6 ++++-- 10 files changed, 31 insertions(+), 20 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 0ac8251b8bb..ce34fdfdc58 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -1,4 +1,5 @@ #include "ArrowBlockInputFormat.h" + #if USE_ARROW #include @@ -22,7 +23,7 @@ namespace ErrorCodes } ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_) - : IInputFormat(header_, in_), stream{stream_} + : IInputFormat(header_, in_), stream{stream_}, arrow_column_to_ch_column(std::make_unique()) { } @@ -63,7 +64,7 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; - arrow_column_to_ch_column.arrowTableToCHChunk(res, *table_result, header, "Arrow"); + arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, header, "Arrow"); return res; } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 6a8acc4a118..3bfead93bf1 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -3,7 +3,6 @@ #if USE_ARROW #include -#include namespace arrow { class RecordBatchReader; } namespace arrow::ipc { class RecordBatchFileReader; } @@ -12,6 +11,7 @@ namespace DB { class ReadBuffer; +class ArrowColumnToCHColumn; class ArrowBlockInputFormat : public IInputFormat { @@ -33,7 +33,7 @@ private: // The following fields are used only for Arrow format std::shared_ptr file_reader; - ArrowColumnToCHColumn arrow_column_to_ch_column; + std::unique_ptr arrow_column_to_ch_column; int record_batch_total = 0; int record_batch_current = 0; diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index d1fdffb700c..9f619320b73 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -18,7 +18,11 @@ namespace ErrorCodes } ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_) - : IOutputFormat(header_, out_), stream{stream_}, format_settings{format_settings_}, arrow_ostream{std::make_shared(out_)} + : IOutputFormat(header_, out_) + , stream{stream_} + , format_settings{format_settings_} + , arrow_ostream{std::make_shared(out_)} + , ch_column_to_arrow_column(std::make_unique()) { } @@ -28,7 +32,7 @@ void ArrowBlockOutputFormat::consume(Chunk chunk) const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; - ch_column_to_arrow_column.chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow", format_settings.arrow.low_cardinality_as_dictionary); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow", format_settings.arrow.low_cardinality_as_dictionary); if (!writer) prepareWriter(arrow_table->schema()); diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h index fc8efe62435..40d81f8b919 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h @@ -4,7 +4,6 @@ #include #include -#include #include "ArrowBufferedStreams.h" namespace arrow { class Schema; } @@ -13,6 +12,8 @@ namespace arrow::ipc { class RecordBatchWriter; } namespace DB { +class CHColumnToArrowColumn; + class ArrowBlockOutputFormat : public IOutputFormat { public: @@ -29,7 +30,7 @@ private: const FormatSettings format_settings; std::shared_ptr arrow_ostream; std::shared_ptr writer; - CHColumnToArrowColumn ch_column_to_arrow_column; + std::unique_ptr ch_column_to_arrow_column; void prepareWriter(const std::shared_ptr & schema); }; diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 0f8ca728c46..bd427bd62e1 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ } while (false) -ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInputFormat(std::move(header_), in_) +ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInputFormat(std::move(header_), in_), arrow_column_to_ch_column(std::make_unique()) { } @@ -54,7 +54,7 @@ Chunk ORCBlockInputFormat::generate() ++stripe_current; - arrow_column_to_ch_column.arrowTableToCHChunk(res, *table_result, header, "ORC"); + arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, header, "ORC"); return res; } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 5a6cfd1364a..f27685a9884 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -3,12 +3,14 @@ #if USE_ORC #include -#include namespace arrow::adapters::orc { class ORCFileReader; } namespace DB { + +class ArrowColumnToCHColumn; + class ORCBlockInputFormat : public IInputFormat { public: @@ -27,7 +29,7 @@ private: std::unique_ptr file_reader; - ArrowColumnToCHColumn arrow_column_to_ch_column; + std::unique_ptr arrow_column_to_ch_column; int stripe_total = 0; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index ea7d35f5bbe..c0d9e330df2 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -31,7 +31,7 @@ namespace ErrorCodes } while (false) ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_) - : IInputFormat(std::move(header_), in_) + : IInputFormat(std::move(header_), in_), arrow_column_to_ch_column(std::make_unique()) { } @@ -54,7 +54,7 @@ Chunk ParquetBlockInputFormat::generate() ++row_group_current; - arrow_column_to_ch_column.arrowTableToCHChunk(res, table, header, "Parquet"); + arrow_column_to_ch_column->arrowTableToCHChunk(res, table, header, "Parquet"); return res; } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index b27bafe04bf..b68f97c005a 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -4,7 +4,6 @@ #if USE_PARQUET #include -#include namespace parquet::arrow { class FileReader; } @@ -13,6 +12,8 @@ namespace arrow { class Buffer; } namespace DB { +class ArrowColumnToCHColumn; + class ParquetBlockInputFormat : public IInputFormat { public: @@ -33,7 +34,7 @@ private: int row_group_total = 0; // indices of columns to read from Parquet file std::vector column_indices; - ArrowColumnToCHColumn arrow_column_to_ch_column; + std::unique_ptr arrow_column_to_ch_column; int row_group_current = 0; }; diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 1eab69239ca..96ef6702cc4 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes } ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) - : IOutputFormat(header_, out_), format_settings{format_settings_} + : IOutputFormat(header_, out_), format_settings{format_settings_}, ch_column_to_arrow_column(std::make_unique()) { } @@ -35,7 +35,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; - ch_column_to_arrow_column.chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet"); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet"); if (!file_writer) { diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index deb011e0274..8114d1ab494 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -4,7 +4,6 @@ #if USE_PARQUET # include # include -# include namespace arrow { @@ -22,6 +21,9 @@ namespace arrow namespace DB { + +class CHColumnToArrowColumn; + class ParquetBlockOutputFormat : public IOutputFormat { public: @@ -37,7 +39,7 @@ private: const FormatSettings format_settings; std::unique_ptr file_writer; - CHColumnToArrowColumn ch_column_to_arrow_column; + std::unique_ptr ch_column_to_arrow_column; }; } From 57a9659c681f921cbbb7409f6a08bacea453bc18 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 3 Jun 2021 16:03:22 +0300 Subject: [PATCH 13/43] Add Arrow/Parquet/ORC in perf tests --- tests/performance/parse_engine_file.xml | 3 +++ tests/performance/select_format.xml | 2 ++ 2 files changed, 5 insertions(+) diff --git a/tests/performance/parse_engine_file.xml b/tests/performance/parse_engine_file.xml index 2740b680b67..2b67c19a4f6 100644 --- a/tests/performance/parse_engine_file.xml +++ b/tests/performance/parse_engine_file.xml @@ -22,6 +22,9 @@ Native Avro MsgPack + ORC + Parquet + Arrow diff --git a/tests/performance/select_format.xml b/tests/performance/select_format.xml index 985ec0f2b52..982039102d0 100644 --- a/tests/performance/select_format.xml +++ b/tests/performance/select_format.xml @@ -36,6 +36,8 @@ Avro MsgPack ORC + Parquet + Arrow From 931e05ab047231fa0af681f941a278b04790cf40 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 7 Jun 2021 18:15:58 +0300 Subject: [PATCH 14/43] Minor refactoring --- .../Formats/Impl/ArrowBlockInputFormat.cpp | 11 +++- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 6 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 59 +++++++++++-------- .../Formats/Impl/ArrowColumnToCHColumn.h | 31 +++++----- .../Formats/Impl/CHColumnToArrowColumn.cpp | 56 ++++++++++-------- .../Formats/Impl/CHColumnToArrowColumn.h | 12 ++-- .../Formats/Impl/ORCBlockInputFormat.cpp | 7 ++- .../Formats/Impl/ParquetBlockInputFormat.cpp | 7 ++- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 6 +- 9 files changed, 117 insertions(+), 78 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index ce34fdfdc58..269faac5258 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -23,14 +23,13 @@ namespace ErrorCodes } ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_) - : IInputFormat(header_, in_), stream{stream_}, arrow_column_to_ch_column(std::make_unique()) + : IInputFormat(header_, in_), stream{stream_} { } Chunk ArrowBlockInputFormat::generate() { Chunk res; - const Block & header = getPort().getHeader(); arrow::Result> batch_result; if (stream) @@ -64,7 +63,7 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, header, "Arrow"); + arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result); return res; } @@ -82,6 +81,8 @@ void ArrowBlockInputFormat::resetParser() void ArrowBlockInputFormat::prepareReader() { + std::shared_ptr schema; + if (stream) { auto stream_reader_status = arrow::ipc::RecordBatchStreamReader::Open(std::make_unique(in)); @@ -89,6 +90,7 @@ void ArrowBlockInputFormat::prepareReader() throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while opening a table: {}", stream_reader_status.status().ToString()); stream_reader = *stream_reader_status; + schema = stream_reader->schema(); } else { @@ -97,8 +99,11 @@ void ArrowBlockInputFormat::prepareReader() throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Error while opening a table: {}", file_reader_status.status().ToString()); file_reader = *file_reader_status; + schema = file_reader->schema(); } + arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), std::move(schema), "Arrow"); + if (stream) record_batch_total = -1; else diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 9f619320b73..42679298e07 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -22,7 +22,6 @@ ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & , stream{stream_} , format_settings{format_settings_} , arrow_ostream{std::make_shared(out_)} - , ch_column_to_arrow_column(std::make_unique()) { } @@ -32,10 +31,13 @@ void ArrowBlockOutputFormat::consume(Chunk chunk) const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; - ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow", format_settings.arrow.low_cardinality_as_dictionary); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); if (!writer) + { prepareWriter(arrow_table->schema()); + ch_column_to_arrow_column = std::make_unique(header, "Arrow", format_settings.arrow.low_cardinality_as_dictionary); + } // TODO: calculate row_group_size depending on a number of rows and table size auto status = writer->WriteTable(*arrow_table, format_settings.arrow.row_group_size); diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 2dad3a15026..856915acf3c 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB @@ -159,11 +160,11 @@ namespace DB if (days_num > DATE_LUT_MAX_DAY_NUM) { // TODO: will it rollback correctly? - throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + internal_column.getName() - + "\" is greater than " - "max allowed Date value, which is " - + std::to_string(DATE_LUT_MAX_DAY_NUM), - ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; + throw Exception + { + fmt::format("Input value {} of a column \"{}\" is greater than max allowed Date value, which is {}", days_num, internal_column.getName(), DATE_LUT_MAX_DAY_NUM), + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE + }; } column_data.emplace_back(days_num); @@ -284,7 +285,7 @@ namespace DB FOR_ARROW_INDEXES_TYPES(DISPATCH) # undef DISPATCH default: - throw Exception("Unsupported type for indexes in LowCardinality: " + arrow_column->type()->name() + ".", ErrorCodes::BAD_ARGUMENTS); + throw Exception(fmt::format("Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()), ErrorCodes::BAD_ARGUMENTS); } } @@ -292,7 +293,7 @@ namespace DB std::shared_ptr & arrow_column, IColumn & internal_column, const std::string & column_name, - const std::string format_name, + const std::string & format_name, bool is_nullable, std::unordered_map dictionary_values) { @@ -310,7 +311,7 @@ namespace DB { throw Exception { - "Can not insert NULL data into non-nullable column \"" + column_name + "\"", + fmt::format("Can not insert NULL data into non-nullable column \"{}\".", column_name), ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN }; } @@ -335,7 +336,6 @@ namespace DB fillColumnWithTimestampData(arrow_column, internal_column); break; case arrow::Type::DECIMAL: - //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/); break; case arrow::Type::MAP: [[fallthrough]]; @@ -428,8 +428,7 @@ namespace DB default: throw Exception { - "Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" - + column_name + "\"", + fmt::format("Unsupported {} type \"{}\" of an input column \"{}\".", format_name, arrow_column->type()->name(), column_name), ErrorCodes::UNKNOWN_TYPE }; } @@ -456,7 +455,7 @@ namespace DB const DataTypeArray * array_type = typeid_cast(column_type.get()); if (!array_type) - throw Exception{"Cannot convert arrow LIST type to a not Array ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + throw Exception{fmt::format("Cannot convert arrow LIST type to a not Array ClickHouse type {}.", column_type->getName()), ErrorCodes::CANNOT_CONVERT_TYPE}; return std::make_shared(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name)); } @@ -466,7 +465,7 @@ namespace DB const auto * struct_type = static_cast(arrow_type.get()); const DataTypeTuple * tuple_type = typeid_cast(column_type.get()); if (!tuple_type) - throw Exception{"Cannot convert arrow STRUCT type to a not Tuple ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + throw Exception{fmt::format("Cannot convert arrow STRUCT type to a not Tuple ClickHouse type {}.", column_type->getName()), ErrorCodes::CANNOT_CONVERT_TYPE}; const DataTypes & tuple_nested_types = tuple_type->getElements(); int internal_fields_num = tuple_nested_types.size(); @@ -474,8 +473,11 @@ namespace DB if (internal_fields_num > struct_type->num_fields()) throw Exception { - "Cannot convert arrow STRUCT with " + std::to_string(struct_type->num_fields()) + " fields to a ClickHouse Tuple with " - + std::to_string(internal_fields_num) + " elements " + column_type->getName(), + fmt::format( + "Cannot convert arrow STRUCT with {} fields to a ClickHouse Tuple with {} elements: {}.", + struct_type->num_fields(), + internal_fields_num, + column_type->getName()), ErrorCodes::CANNOT_CONVERT_TYPE }; @@ -500,7 +502,7 @@ namespace DB const auto * arrow_map_type = typeid_cast(arrow_type.get()); const auto * map_type = typeid_cast(column_type.get()); if (!map_type) - throw Exception{"Cannot convert arrow MAP type to a not Map ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + throw Exception{fmt::format("Cannot convert arrow MAP type to a not Map ClickHouse type {}.", column_type->getName()), ErrorCodes::CANNOT_CONVERT_TYPE}; return std::make_shared( getInternalType(arrow_map_type->key_type(), map_type->getKeyType(), column_name, format_name), @@ -516,13 +518,24 @@ namespace DB } throw Exception { - "The type \"" + arrow_type->name() + "\" of an input column \"" + column_name + "\" is not supported for conversion from a " + format_name + " data format", + fmt::format("The type \"{}\" of an input column \"{}\" is not supported for conversion from a {} data format.", arrow_type->name(), column_name, format_name), ErrorCodes::CANNOT_CONVERT_TYPE }; } - void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, - const Block & header, std::string format_name) + ArrowColumnToCHColumn::ArrowColumnToCHColumn(const Block & header_, std::shared_ptr schema_, const std::string & format_name_) : header(header_), format_name(format_name_) + { + for (const auto & field : schema_->fields()) + { + if (header.has(field->name())) + { + const auto column_type = recursiveRemoveLowCardinality(header.getByName(field->name()).type); + name_to_internal_type[field->name()] = getInternalType(field->type(), column_type, field->name(), format_name); + } + } + } + + void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) { Columns columns_list; UInt64 num_rows = 0; @@ -540,18 +553,16 @@ namespace DB for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) { - ColumnWithTypeAndName header_column = header.getByPosition(column_i); - const auto column_type = recursiveRemoveLowCardinality(header_column.type); + const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); if (name_to_column_ptr.find(header_column.name) == name_to_column_ptr.end()) // TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable? - throw Exception{"Column \"" + header_column.name + "\" is not presented in input data", + throw Exception{fmt::format("Column \"{}\" is not presented in input data.", header_column.name), ErrorCodes::THERE_IS_NO_COLUMN}; std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; - DataTypePtr internal_type = getInternalType(arrow_column->type(), column_type, header_column.name, format_name); - + DataTypePtr & internal_type = name_to_internal_type[header_column.name]; MutableColumnPtr read_column = internal_type->createColumn(); readColumnFromArrowColumn(arrow_column, *read_column, header_column.name, format_name, false, dictionary_values); diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 0ba2d891a2c..7da54a8a02d 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -19,11 +19,15 @@ namespace DB { - class ArrowColumnToCHColumn - { - private: +class ArrowColumnToCHColumn +{ +public: + ArrowColumnToCHColumn(const Block & header_, std::shared_ptr schema_, const std::string & format_name_); -# define FOR_ARROW_NUMERIC_TYPES(M) \ + void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); + +private: +#define FOR_ARROW_NUMERIC_TYPES(M) \ M(arrow::Type::UINT8, DB::UInt8) \ M(arrow::Type::INT8, DB::Int8) \ M(arrow::Type::UINT16, DB::UInt16) \ @@ -36,7 +40,7 @@ namespace DB M(arrow::Type::FLOAT, DB::Float32) \ M(arrow::Type::DOUBLE, DB::Float64) -# define FOR_ARROW_INDEXES_TYPES(M) \ +#define FOR_ARROW_INDEXES_TYPES(M) \ M(arrow::Type::UINT8, DB::UInt8) \ M(arrow::Type::INT8, DB::UInt8) \ M(arrow::Type::UINT16, DB::UInt16) \ @@ -46,15 +50,14 @@ namespace DB M(arrow::Type::UINT64, DB::UInt64) \ M(arrow::Type::INT64, DB::UInt64) - /// Map {column name : dictionary column}. - /// To avoid converting dictionary from Arrow Dictionary - /// to LowCardinality every chunk we save it and reuse. - std::unordered_map dictionary_values; - public: - - void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, - const Block & header, std::string format_name); - }; + const Block & header; + std::unordered_map name_to_internal_type; + const std::string format_name; + /// Map {column name : dictionary column}. + /// To avoid converting dictionary from Arrow Dictionary + /// to LowCardinality every chunk we save it and reuse. + std::unordered_map dictionary_values; +}; } #endif diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index c99c80c8d84..01fc80ad950 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -60,7 +60,7 @@ namespace DB static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) { if (!status.ok()) - throw Exception{"Error with a " + format_name + " column \"" + column_name + "\": " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; + throw Exception{fmt::format("Error with a {} column \"{}\": {}.", format_name, column_name, status.ToString()), ErrorCodes::UNKNOWN_EXCEPTION}; } template @@ -191,7 +191,7 @@ namespace DB case TypeIndex::UInt64: return extractIndexesImpl(column, start, end); default: - throw Exception("Indexes column must be ColumnUInt, got " + column->getName(), + throw Exception(fmt::format("Indexes column must be ColumnUInt, got {}.", column->getName()), ErrorCodes::LOGICAL_ERROR); } } @@ -439,7 +439,7 @@ namespace DB { throw Exception { - "Internal type \"" + column_type_name + "\" of a column \"" + column_name + "\" is not supported for conversion into a " + format_name + " data format", + fmt::format("Internal type \"{}\" of a column \"{}\" is not supported for conversion into a {} data format.", column_type_name, column_name, format_name), ErrorCodes::UNKNOWN_TYPE }; } @@ -486,7 +486,7 @@ namespace DB case TypeIndex::UInt64: return arrow::int64(); default: - throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got " + indexes_column->getName(), + throw Exception(fmt::format("Indexes column for getUniqueIndex must be ColumnUInt, got {}.", indexes_column->getName()), ErrorCodes::LOGICAL_ERROR); } } @@ -580,51 +580,61 @@ namespace DB return arrow_type_it->second; } - throw Exception{"The type \"" + column_type->getName() + "\" of a column \"" + column_name + "\"" - " is not supported for conversion into a " + format_name + " data format", + throw Exception{fmt::format("The type \"{}\" of a column \"{}\" is not supported for conversion into a {} data format.", column_type->getName(), column_name, format_name), ErrorCodes::UNKNOWN_TYPE}; } + CHColumnToArrowColumn::CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_) + : format_name(format_name_), low_cardinality_as_dictionary(low_cardinality_as_dictionary_) + { + arrow_fields.reserve(header.columns()); + header_columns.reserve(header.columns()); + for (auto column : header.getColumnsWithTypeAndName()) + { + if (!low_cardinality_as_dictionary) + { + column.type = recursiveRemoveLowCardinality(column.type); + column.column = recursiveRemoveLowCardinality(column.column); + } + bool is_column_nullable = false; + auto arrow_type = getArrowType(column.type, column.column, column.name, format_name, &is_column_nullable); + arrow_fields.emplace_back(std::make_shared(column.name, arrow_type, is_column_nullable)); + header_columns.emplace_back(std::move(column)); + } + } + void CHColumnToArrowColumn::chChunkToArrowTable( std::shared_ptr & res, - const Block & header, const Chunk & chunk, - size_t columns_num, - String format_name, - bool low_cardinality_as_dictionary) + size_t columns_num) { /// For arrow::Schema and arrow::Table creation - std::vector> arrow_fields; std::vector> arrow_arrays; arrow_fields.reserve(columns_num); arrow_arrays.reserve(columns_num); for (size_t column_i = 0; column_i < columns_num; ++column_i) { - // TODO: constructed every iteration - ColumnWithTypeAndName column = header.safeGetByPosition(column_i); - column.column = chunk.getColumns()[column_i]; + const ColumnWithTypeAndName & header_column = header_columns[column_i]; + auto column = chunk.getColumns()[column_i]; if (!low_cardinality_as_dictionary) - { - column.column = recursiveRemoveLowCardinality(column.column); - column.type = recursiveRemoveLowCardinality(column.type); - } + column = recursiveRemoveLowCardinality(column); bool is_column_nullable = false; - auto arrow_type = getArrowType(column.type, column.column, column.name, format_name, &is_column_nullable); - arrow_fields.emplace_back(std::make_shared(column.name, arrow_type, is_column_nullable)); + auto arrow_type = getArrowType(header_column.type, column, header_column.name, format_name, &is_column_nullable); + arrow_fields.emplace_back(std::make_shared(header_column.name, arrow_type, is_column_nullable)); arrow::MemoryPool* pool = arrow::default_memory_pool(); std::unique_ptr array_builder; arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder); - checkStatus(status, column.column->getName(), format_name); + checkStatus(status, column->getName(), format_name); - fillArrowArray(column.name, column.column, column.type, nullptr, array_builder.get(), format_name, 0, column.column->size(), dictionary_values); + fillArrowArray(header_column.name, column, header_column.type, nullptr, array_builder.get(), format_name, 0, column->size(), dictionary_values); std::shared_ptr arrow_array; status = array_builder->Finish(&arrow_array); - checkStatus(status, column.column->getName(), format_name); + checkStatus(status, column->getName(), format_name); arrow_arrays.emplace_back(std::move(arrow_array)); } diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h index 9df70b5bceb..efe02a0d7d9 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h @@ -12,6 +12,10 @@ namespace DB class CHColumnToArrowColumn { +public: + CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_ = false); + + void chChunkToArrowTable(std::shared_ptr & res, const Chunk & chunk, size_t columns_num); private: #define FOR_INTERNAL_NUMERIC_TYPES(M) \ @@ -39,14 +43,14 @@ private: M(DOUBLE, arrow::DoubleType) \ M(STRING, arrow::StringType) + ColumnsWithTypeAndName header_columns; + std::vector> arrow_fields; + const std::string format_name; + bool low_cardinality_as_dictionary; /// Map {column name : arrow dictionary}. /// To avoid converting dictionary from LowCardinality to Arrow /// Dictionary every chunk we save it and reuse. std::unordered_map> dictionary_values; - -public: - void chChunkToArrowTable(std::shared_ptr & res, const Block & header, const Chunk & chunk, - size_t columns_num, String format_name, bool low_cardinality_as_dictionary = false); }; } #endif diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index bd427bd62e1..6ee247413e9 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -26,14 +26,13 @@ namespace ErrorCodes throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ } while (false) -ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInputFormat(std::move(header_), in_), arrow_column_to_ch_column(std::make_unique()) +ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInputFormat(std::move(header_), in_) { } Chunk ORCBlockInputFormat::generate() { Chunk res; - const Block & header = getPort().getHeader(); if (!file_reader) prepareReader(); @@ -54,7 +53,7 @@ Chunk ORCBlockInputFormat::generate() ++stripe_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, header, "ORC"); + arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result); return res; } @@ -99,6 +98,8 @@ void ORCBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); + arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), schema, "ORC"); + /// In ReadStripe column indices should be started from 1, /// because 0 indicates to select all columns. int index = 1; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c0d9e330df2..07a0e15cb6b 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -31,14 +31,13 @@ namespace ErrorCodes } while (false) ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_) - : IInputFormat(std::move(header_), in_), arrow_column_to_ch_column(std::make_unique()) + : IInputFormat(std::move(header_), in_) { } Chunk ParquetBlockInputFormat::generate() { Chunk res; - const Block & header = getPort().getHeader(); if (!file_reader) prepareReader(); @@ -54,7 +53,7 @@ Chunk ParquetBlockInputFormat::generate() ++row_group_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, table, header, "Parquet"); + arrow_column_to_ch_column->arrowTableToCHChunk(res, table); return res; } @@ -99,6 +98,8 @@ void ParquetBlockInputFormat::prepareReader() std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); + arrow_column_to_ch_column = std::make_unique(getPort().getHeader(), schema, "Parquet"); + int index = 0; for (int i = 0; i < schema->num_fields(); ++i) { diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 96ef6702cc4..c53ee44de36 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes } ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) - : IOutputFormat(header_, out_), format_settings{format_settings_}, ch_column_to_arrow_column(std::make_unique()) + : IOutputFormat(header_, out_), format_settings{format_settings_} { } @@ -35,7 +35,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; - ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet"); + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); if (!file_writer) { @@ -54,6 +54,8 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) &file_writer); if (!status.ok()) throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; + + ch_column_to_arrow_column = std::make_unique(header, "Parquet"); } // TODO: calculate row_group_size depending on a number of rows and table size From 692150b578737f1230d7fd3fdaf6b9a11639703e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 8 Jun 2021 13:37:54 +0300 Subject: [PATCH 15/43] Fix tests --- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 11 +++++++---- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 8 +------- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 9 ++++++--- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 42679298e07..8f43d03de38 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -27,17 +27,20 @@ ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & void ArrowBlockOutputFormat::consume(Chunk chunk) { - const Block & header = getPort(PortKind::Main).getHeader(); const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; + if (!ch_column_to_arrow_column) + { + const Block & header = getPort(PortKind::Main).getHeader(); + ch_column_to_arrow_column + = std::make_unique(header, "Arrow", format_settings.arrow.low_cardinality_as_dictionary); + } + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); if (!writer) - { prepareWriter(arrow_table->schema()); - ch_column_to_arrow_column = std::make_unique(header, "Arrow", format_settings.arrow.low_cardinality_as_dictionary); - } // TODO: calculate row_group_size depending on a number of rows and table size auto status = writer->WriteTable(*arrow_table, format_settings.arrow.row_group_size); diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 01fc80ad950..4dbb95b327a 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -610,9 +610,7 @@ namespace DB { /// For arrow::Schema and arrow::Table creation std::vector> arrow_arrays; - arrow_fields.reserve(columns_num); arrow_arrays.reserve(columns_num); - for (size_t column_i = 0; column_i < columns_num; ++column_i) { const ColumnWithTypeAndName & header_column = header_columns[column_i]; @@ -621,10 +619,6 @@ namespace DB if (!low_cardinality_as_dictionary) column = recursiveRemoveLowCardinality(column); - bool is_column_nullable = false; - auto arrow_type = getArrowType(header_column.type, column, header_column.name, format_name, &is_column_nullable); - arrow_fields.emplace_back(std::make_shared(header_column.name, arrow_type, is_column_nullable)); - arrow::MemoryPool* pool = arrow::default_memory_pool(); std::unique_ptr array_builder; arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder); @@ -638,7 +632,7 @@ namespace DB arrow_arrays.emplace_back(std::move(arrow_array)); } - std::shared_ptr arrow_schema = std::make_shared(std::move(arrow_fields)); + std::shared_ptr arrow_schema = std::make_shared(arrow_fields); res = arrow::Table::Make(arrow_schema, arrow_arrays); } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index c53ee44de36..800fd0ff0e8 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -31,10 +31,15 @@ ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Blo void ParquetBlockOutputFormat::consume(Chunk chunk) { - const Block & header = getPort(PortKind::Main).getHeader(); const size_t columns_num = chunk.getNumColumns(); std::shared_ptr arrow_table; + if (!ch_column_to_arrow_column) + { + const Block & header = getPort(PortKind::Main).getHeader(); + ch_column_to_arrow_column = std::make_unique(header, "Parquet"); + } + ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num); if (!file_writer) @@ -54,8 +59,6 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) &file_writer); if (!status.ok()) throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; - - ch_column_to_arrow_column = std::make_unique(header, "Parquet"); } // TODO: calculate row_group_size depending on a number of rows and table size From 313b0a8ad573367ed382065e0fb95ec86e6c9cc8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 8 Jun 2021 14:00:12 +0300 Subject: [PATCH 16/43] Fix test --- .../0_stateless/data_parquet/datapage_v2.snappy.parquet.columns | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/data_parquet/datapage_v2.snappy.parquet.columns b/tests/queries/0_stateless/data_parquet/datapage_v2.snappy.parquet.columns index d9e51028f22..c6bb5057cc2 100644 --- a/tests/queries/0_stateless/data_parquet/datapage_v2.snappy.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/datapage_v2.snappy.parquet.columns @@ -1 +1 @@ -`a` Nullable(String), `b` Nullable(Int32), `c` Nullable(Float64), `d` Nullable(UInt8), `e` Nullable(Int32) +`a` Nullable(String), `b` Array(Nullable(Int32)), `c` Nullable(Float64), `d` Nullable(UInt8), `e` Array(Nullable(Int32)) From 48cd8f2207c60a9f7498862d0d8c2d8285323881 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 11 Jun 2021 14:33:05 +0300 Subject: [PATCH 17/43] Fix build --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 4 ++-- src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 856915acf3c..edf131cd49e 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -428,7 +428,7 @@ namespace DB default: throw Exception { - fmt::format("Unsupported {} type \"{}\" of an input column \"{}\".", format_name, arrow_column->type()->name(), column_name), + fmt::format(R"(Unsupported {} type "{}" of an input column "{}".)", format_name, arrow_column->type()->name(), column_name), ErrorCodes::UNKNOWN_TYPE }; } @@ -518,7 +518,7 @@ namespace DB } throw Exception { - fmt::format("The type \"{}\" of an input column \"{}\" is not supported for conversion from a {} data format.", arrow_type->name(), column_name, format_name), + fmt::format(R"(The type "{}" of an input column "{}" is not supported for conversion from a {} data format.)", arrow_type->name(), column_name, format_name), ErrorCodes::CANNOT_CONVERT_TYPE }; } diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 4dbb95b327a..cc487535e37 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -439,7 +439,7 @@ namespace DB { throw Exception { - fmt::format("Internal type \"{}\" of a column \"{}\" is not supported for conversion into a {} data format.", column_type_name, column_name, format_name), + fmt::format(R"(Internal type "{}" of a column "{}" is not supported for conversion into a {} data format.)", column_type_name, column_name, format_name), ErrorCodes::UNKNOWN_TYPE }; } @@ -580,7 +580,7 @@ namespace DB return arrow_type_it->second; } - throw Exception{fmt::format("The type \"{}\" of a column \"{}\" is not supported for conversion into a {} data format.", column_type->getName(), column_name, format_name), + throw Exception{fmt::format(R"(The type "{}" of a column "{}" is not supported for conversion into a {} data format.)", column_type->getName(), column_name, format_name), ErrorCodes::UNKNOWN_TYPE}; } From 5c6bda62946db6c732f9eee8e3c34a0ad99ce4ae Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 16 Jun 2021 15:57:36 +0300 Subject: [PATCH 18/43] Remove useless srcs --- contrib/arrow-cmake/CMakeLists.txt | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 878b46b39a6..cf4dec16119 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -284,18 +284,6 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/utf8.cc" "${LIBRARY_DIR}/util/value_parsing.cc" - "${LIBRARY_DIR}/vendored/base64.cpp" - "${LIBRARY_DIR}/vendored/datetime/tz.cpp" - "${LIBRARY_DIR}/vendored/double-conversion/bignum.cc" - "${LIBRARY_DIR}/vendored/double-conversion/double-conversion.cc" - "${LIBRARY_DIR}/vendored/double-conversion/bignum-dtoa.cc" - "${LIBRARY_DIR}/vendored/double-conversion/fast-dtoa.cc" - "${LIBRARY_DIR}/vendored/double-conversion/cached-powers.cc" - "${LIBRARY_DIR}/vendored/double-conversion/fixed-dtoa.cc" - "${LIBRARY_DIR}/vendored/double-conversion/diy-fp.cc" - "${LIBRARY_DIR}/vendored/double-conversion/strtod.cc" - - "${LIBRARY_DIR}/c/bridge.cc" ${ORC_SRCS} ) From 81998a5c6f9c952a41f7635a670e1530f5d12d79 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 16 Jun 2021 16:12:23 +0300 Subject: [PATCH 19/43] Fix --- contrib/arrow-cmake/CMakeLists.txt | 1 + contrib/ryu | 1 + 2 files changed, 2 insertions(+) create mode 160000 contrib/ryu diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index cf4dec16119..2237be9913a 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -284,6 +284,7 @@ set(ARROW_SRCS "${LIBRARY_DIR}/util/utf8.cc" "${LIBRARY_DIR}/util/value_parsing.cc" + "${LIBRARY_DIR}/vendored/base64.cpp" ${ORC_SRCS} ) diff --git a/contrib/ryu b/contrib/ryu new file mode 160000 index 00000000000..5b4a853534b --- /dev/null +++ b/contrib/ryu @@ -0,0 +1 @@ +Subproject commit 5b4a853534b47438b4d97935370f6b2397137c2b From 83919363eae17728f81513412d413cfdded9df8b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 16 Jun 2021 16:55:34 +0300 Subject: [PATCH 20/43] Fix broken contrib --- contrib/ryu | 1 - 1 file changed, 1 deletion(-) delete mode 160000 contrib/ryu diff --git a/contrib/ryu b/contrib/ryu deleted file mode 160000 index 5b4a853534b..00000000000 --- a/contrib/ryu +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 5b4a853534b47438b4d97935370f6b2397137c2b From 87cb617f3d66f47010a108ae10c8735ea1713ae0 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 17 Jun 2021 12:42:01 +0300 Subject: [PATCH 21/43] Turn off WITH_COVERAGE in debug special build --- tests/ci/ci_config.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 6fa1f215e0d..39722e17f25 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -111,7 +111,7 @@ "bundled": "bundled", "splitted": "unsplitted", "tidy": "enable", - "with_coverage": true + "with_coverage": false }, { "compiler": "clang-11", From 8a5bc2e7bc3515885ce2624d2ffcf4abe4c9cad2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jun 2021 12:23:22 +0300 Subject: [PATCH 22/43] Add run-id option to integration tests --- tests/integration/ci-runner.py | 4 ++-- tests/integration/conftest.py | 9 ++++++++- tests/integration/helpers/cluster.py | 9 ++++++++- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 8b8462a7125..0af76fe2648 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -377,8 +377,8 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( - repo_path, image_cmd, test_cmd, parallel_cmd, _get_deselect_option(self.should_skip_tests()), output_path) + cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-ss -rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( + repo_path, image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: logging.info("Executing cmd: %s", cmd) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index fa14e2b06d6..890b329673c 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -28,4 +28,11 @@ def cleanup_environment(): logging.exception(f"cleanup_environment:{str(e)}") pass - yield \ No newline at end of file + yield + + +def pytest_addoption(parser): + parser.addoption("--run-id", default="", help="run-id is used as postfix in _instances_{} directory") + +def pytest_configure(config): + os.environ['INTEGRATION_TESTS_RUN_ID'] = config.option.run_id diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 27d8c05d8ed..dbe7649b151 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -203,7 +203,14 @@ class ClickHouseCluster: project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name # docker-compose removes everything non-alphanumeric from project names so we do it too. self.project_name = re.sub(r'[^a-z0-9]', '', project_name.lower()) - self.instances_dir = p.join(self.base_dir, '_instances' + ('' if not self.name else '_' + self.name)) + instances_dir_name = '_instances' + if self.name: + instances_dir_name += '_' + self.name + + if 'INTEGRATION_TESTS_RUN_ID' in os.environ: + instances_dir_name += '_' + os.environ['INTEGRATION_TESTS_RUN_ID'] + + self.instances_dir = p.join(self.base_dir, instances_dir_name) self.docker_logs_path = p.join(self.instances_dir, 'docker.log') self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_variables = {} From f0490ad7c19dbb96e2c98583a60dfe7c702e6f62 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jun 2021 12:24:04 +0300 Subject: [PATCH 23/43] Better formatting --- tests/integration/conftest.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 890b329673c..993e7a6e973 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -30,7 +30,6 @@ def cleanup_environment(): yield - def pytest_addoption(parser): parser.addoption("--run-id", default="", help="run-id is used as postfix in _instances_{} directory") From 49cd16e74ff496d5c63fa0051090dbb53ddb631d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jun 2021 22:08:12 +0300 Subject: [PATCH 24/43] Better --- tests/integration/helpers/cluster.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dbe7649b151..32edd8892ac 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -208,7 +208,7 @@ class ClickHouseCluster: instances_dir_name += '_' + self.name if 'INTEGRATION_TESTS_RUN_ID' in os.environ: - instances_dir_name += '_' + os.environ['INTEGRATION_TESTS_RUN_ID'] + instances_dir_name += '_' + shlex.quote(os.environ['INTEGRATION_TESTS_RUN_ID']) self.instances_dir = p.join(self.base_dir, instances_dir_name) self.docker_logs_path = p.join(self.instances_dir, 'docker.log') @@ -428,7 +428,15 @@ class ClickHouseCluster: pass def get_docker_handle(self, docker_id): - return self.docker_client.containers.get(docker_id) + exception = None + for i in range(5): + try: + return self.docker_client.containers.get(docker_id) + except Exception as ex: + print("Got exception getting docker handle", str(ex)) + time.sleep(i * 2) + exception = ex + raise exception def get_client_cmd(self): cmd = self.client_bin_path From 6567b819bd3128ef202bc2233db6d8de297eea5e Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 18 Jun 2021 17:29:33 -0400 Subject: [PATCH 25/43] Enabling TestFlows RBAC tests. --- tests/testflows/rbac/regression.py | 7 +++++++ tests/testflows/regression.py | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/testflows/rbac/regression.py b/tests/testflows/rbac/regression.py index 549ccdf80d8..145865b2fa9 100755 --- a/tests/testflows/rbac/regression.py +++ b/tests/testflows/rbac/regression.py @@ -30,6 +30,7 @@ issue_18110 = "https://github.com/ClickHouse/ClickHouse/issues/18110" issue_18206 = "https://github.com/ClickHouse/ClickHouse/issues/18206" issue_21083 = "https://github.com/ClickHouse/ClickHouse/issues/21083" issue_21084 = "https://github.com/ClickHouse/ClickHouse/issues/21084" +issue_25413 = "https://github.com/ClickHouse/ClickHouse/issues/25413" xfails = { "syntax/show create quota/I show create quota current": @@ -144,6 +145,12 @@ xfails = { [(Fail, "new bug")], "privileges/show dictionaries/:/check privilege/check privilege=DROP DICTIONARY/show dict/SHOW DICTIONARIES with privilege": [(Fail, "new bug")], + "privileges/kill mutation/:/:/KILL ALTER : without privilege": + [(Fail, issue_25413)], + "privileges/kill mutation/:/:/KILL ALTER : with revoked privilege": + [(Fail, issue_25413)], + "privileges/kill mutation/:/:/KILL ALTER : with revoked ALL privilege": + [(Fail, issue_25413)] } xflags = { diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index 5ad529c1b5a..c2e143a4b1c 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -24,7 +24,7 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): try: run_scenario(pool, tasks, Feature(test=load("example.regression", "regression")), args) run_scenario(pool, tasks, Feature(test=load("ldap.regression", "regression")), args) - #run_scenario(pool, tasks, Feature(test=load("rbac.regression", "regression")), args) + run_scenario(pool, tasks, Feature(test=load("rbac.regression", "regression")), args) run_scenario(pool, tasks, Feature(test=load("aes_encryption.regression", "regression")), args) run_scenario(pool, tasks, Feature(test=load("map_type.regression", "regression")), args) run_scenario(pool, tasks, Feature(test=load("window_functions.regression", "regression")), args) From 8615e31c88bf0085392d239f53be9a13d56817f3 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 18 Jun 2021 18:35:35 -0400 Subject: [PATCH 26/43] Fixing extended precision data types SRS name. --- tests/testflows/extended_precision_data_types/regression.py | 2 +- .../requirements/requirements.md | 2 +- .../requirements/requirements.py | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/testflows/extended_precision_data_types/regression.py b/tests/testflows/extended_precision_data_types/regression.py index a0c3186c961..8fea6f68e5c 100755 --- a/tests/testflows/extended_precision_data_types/regression.py +++ b/tests/testflows/extended_precision_data_types/regression.py @@ -22,7 +22,7 @@ xflags = { @XFlags(xflags) @Name("extended precision data types") @Specifications( - QA_SRS020_ClickHouse_Extended_Precision_Data_Types + SRS020_ClickHouse_Extended_Precision_Data_Types ) @Requirements( RQ_SRS_020_ClickHouse_Extended_Precision("1.0"), diff --git a/tests/testflows/extended_precision_data_types/requirements/requirements.md b/tests/testflows/extended_precision_data_types/requirements/requirements.md index 9bbd59a14d5..232eb4d7aba 100644 --- a/tests/testflows/extended_precision_data_types/requirements/requirements.md +++ b/tests/testflows/extended_precision_data_types/requirements/requirements.md @@ -1,4 +1,4 @@ -# QA-SRS020 ClickHouse Extended Precision Data Types +# SRS020 ClickHouse Extended Precision Data Types # Software Requirements Specification ## Table of Contents diff --git a/tests/testflows/extended_precision_data_types/requirements/requirements.py b/tests/testflows/extended_precision_data_types/requirements/requirements.py index 3fcf7798651..6069b98a551 100644 --- a/tests/testflows/extended_precision_data_types/requirements/requirements.py +++ b/tests/testflows/extended_precision_data_types/requirements/requirements.py @@ -754,8 +754,8 @@ RQ_SRS_020_ClickHouse_Extended_Precision_Create_Table = Requirement( level=3, num='4.13.1') -QA_SRS020_ClickHouse_Extended_Precision_Data_Types = Specification( - name='QA-SRS020 ClickHouse Extended Precision Data Types', +SRS020_ClickHouse_Extended_Precision_Data_Types = Specification( + name='SRS020 ClickHouse Extended Precision Data Types', description=None, author=None, date=None, @@ -855,7 +855,7 @@ QA_SRS020_ClickHouse_Extended_Precision_Data_Types = Specification( RQ_SRS_020_ClickHouse_Extended_Precision_Create_Table, ), content=''' -# QA-SRS020 ClickHouse Extended Precision Data Types +# SRS020 ClickHouse Extended Precision Data Types # Software Requirements Specification ## Table of Contents From c6339b4c08ad8f50bf34831367de9a1ea90b1e09 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 May 2021 10:07:25 +0300 Subject: [PATCH 27/43] Stable NOT chain formatting --- src/Parsers/ExpressionListParsers.cpp | 35 ++++++++++--------- .../01920_not_chain_format.reference | 5 +++ .../0_stateless/01920_not_chain_format.sql | 3 ++ 3 files changed, 26 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/01920_not_chain_format.reference create mode 100644 tests/queries/0_stateless/01920_not_chain_format.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index eec79edc05e..e6af11399de 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -489,14 +489,12 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex /** This is done, because among the unary operators there is only a minus and NOT. * But for a minus the chain of unary operators does not need to be supported. */ + size_t count = 1; if (it[0] && 0 == strncmp(it[0], "NOT", 3)) { - /// Was there an even number of NOTs. - bool even = false; - - const char ** jt; while (true) { + const char ** jt; for (jt = operators; *jt; jt += 2) if (parseOperator(pos, *jt, expected)) break; @@ -504,11 +502,8 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex if (!*jt) break; - even = !even; + ++count; } - - if (even) - it = jt; /// Zero the result of parsing the first NOT. It turns out, as if there is no `NOT` chain at all. } ASTPtr elem; @@ -519,19 +514,25 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex node = elem; else { - /// the function corresponding to the operator - auto function = std::make_shared(); + for (size_t i = 0; i < count; ++i) + { + /// the function corresponding to the operator + auto function = std::make_shared(); - /// function arguments - auto exp_list = std::make_shared(); + /// function arguments + auto exp_list = std::make_shared(); - function->name = it[1]; - function->arguments = exp_list; - function->children.push_back(exp_list); + function->name = it[1]; + function->arguments = exp_list; + function->children.push_back(exp_list); - exp_list->children.push_back(elem); + if (node) + exp_list->children.push_back(node); + else + exp_list->children.push_back(elem); - node = function; + node = function; + } } return true; diff --git a/tests/queries/0_stateless/01920_not_chain_format.reference b/tests/queries/0_stateless/01920_not_chain_format.reference new file mode 100644 index 00000000000..22abfd17dc7 --- /dev/null +++ b/tests/queries/0_stateless/01920_not_chain_format.reference @@ -0,0 +1,5 @@ +-- { echo } +EXPLAIN SYNTAX SELECT NOT NOT (NOT (NOT (NULL))); +SELECT NOT (NOT (NOT NOT NULL)) +EXPLAIN SYNTAX SELECT NOT (NOT (NOT NOT NULL)); +SELECT NOT (NOT (NOT NOT NULL)) diff --git a/tests/queries/0_stateless/01920_not_chain_format.sql b/tests/queries/0_stateless/01920_not_chain_format.sql new file mode 100644 index 00000000000..79a5050432d --- /dev/null +++ b/tests/queries/0_stateless/01920_not_chain_format.sql @@ -0,0 +1,3 @@ +-- { echo } +EXPLAIN SYNTAX SELECT NOT NOT (NOT (NOT (NULL))); +EXPLAIN SYNTAX SELECT NOT (NOT (NOT NOT NULL)); From efb6433a391ac2db02085090eaba7ff8ddc597c2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Jun 2021 09:57:39 +0300 Subject: [PATCH 28/43] Fix container-overflow in replxx during incremental search (Ctrl-R) ASAN report: ================================================================= ==7686==ERROR: AddressSanitizer: container-overflow on address 0x6200000bf080 at pc 0x00002a787e79 bp 0x7fffffffa2f0 sp 0x7fffffffa2e8 READ of size 4 at 0x6200000bf080 thread T0 0 0x2a787e78 in replxx::calculate_displayed_length(char32_t const*, int) obj-x86_64-linux-gnu/../contrib/replxx/src/util.cxx:66:15 1 0x2a75786c in replxx::Replxx::ReplxxImpl::dynamicRefresh(replxx::Prompt&, char32_t*, int, int) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:2201:3 2 0x2a7453f0 in replxx::Replxx::ReplxxImpl::incremental_history_search(char32_t) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:2008:3 3 0x2a73eecc in replxx::Replxx::ReplxxImpl::action(unsigned long long, replxx::Replxx::ACTION_RESULT (replxx::Replxx::ReplxxImpl::* const&)(char32_t), char32_t) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:1246:29 4 0x2a73eecc in replxx::Replxx::ReplxxImpl::invoke(replxx::Replxx::ACTION, char32_t) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:318:70 5 0x2a74ed29 in std::__1::__function::__policy_func::operator()(char32_t&&) const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 6 0x2a74ed29 in std::__1::function::operator()(char32_t) const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 7 0x2a74ed29 in replxx::Replxx::ReplxxImpl::get_input_line() obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:1234:11 8 0x2a74dd3c in replxx::Replxx::ReplxxImpl::input(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:580:8 9 0x2a2a4075 in ReplxxLineReader::readOneLine(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../base/common/ReplxxLineReader.cpp:112:29 10 0x2a29b499 in LineReader::readLine(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../base/common/LineReader.cpp:81:26 11 0xb580f02 in DB::Client::mainImpl() obj-x86_64-linux-gnu/../programs/client/Client.cpp:665:33 12 0xb575825 in DB::Client::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) obj-x86_64-linux-gnu/../programs/client/Client.cpp:300:20 13 0x2a3aff25 in Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 14 0xb54c810 in mainEntryClickHouseClient(int, char**) obj-x86_64-linux-gnu/../programs/client/Client.cpp:2702:23 15 0xb326d8a in main obj-x86_64-linux-gnu/../programs/main.cpp:360:12 16 0x7ffff7dcbb24 in __libc_start_main (/usr/lib/libc.so.6+0x27b24) 17 0xb2794ad in _start (/src/ch/tmp/upstream/clickhouse-asan+0xb2794ad) 0x6200000bf080 is located 0 bytes inside of 3672-byte region [0x6200000bf080,0x6200000bfed8) allocated by thread T0 here: 0 0xb3231dd in operator new(unsigned long) (/src/ch/tmp/upstream/clickhouse-asan+0xb3231dd) 1 0x2a75fb15 in void* std::__1::__libcpp_operator_new(unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/new:235:10 2 0x2a75fb15 in std::__1::__libcpp_allocate(unsigned long, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/new:261:10 3 0x2a75fb15 in std::__1::allocator::allocate(unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:840:38 4 0x2a75fb15 in std::__1::allocator_traits >::allocate(std::__1::allocator&, unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:468:21 5 0x2a75fb15 in std::__1::vector >::__vallocate(unsigned long) obj-x86_64-linux-gnu/../contrib/libcxx/include/vector:993:37 6 0x2a75fb15 in std::__1::enable_if<(__is_cpp17_forward_iterator::value) && (is_constructible::reference>::value), void>::type std::__1::vector >::assign(char32_t*, char32_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/vector:1460:9 7 0x2a745242 in std::__1::vector >::operator=(std::__1::vector > const&) obj-x86_64-linux-gnu/../contrib/libcxx/include/vector:1405:9 8 0x2a745242 in replxx::UnicodeString::assign(replxx::UnicodeString const&) obj-x86_64-linux-gnu/../contrib/replxx/src/unicodestring.hxx:83:9 9 0x2a745242 in replxx::Replxx::ReplxxImpl::incremental_history_search(char32_t) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:1993:24 10 0x2a73eecc in replxx::Replxx::ReplxxImpl::action(unsigned long long, replxx::Replxx::ACTION_RESULT (replxx::Replxx::ReplxxImpl::* const&)(char32_t), char32_t) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:1246:29 11 0x2a73eecc in replxx::Replxx::ReplxxImpl::invoke(replxx::Replxx::ACTION, char32_t) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:318:70 12 0x2a74ed29 in std::__1::__function::__policy_func::operator()(char32_t&&) const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 13 0x2a74ed29 in std::__1::function::operator()(char32_t) const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 14 0x2a74ed29 in replxx::Replxx::ReplxxImpl::get_input_line() obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:1234:11 15 0x2a74dd3c in replxx::Replxx::ReplxxImpl::input(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../contrib/replxx/src/replxx_impl.cxx:580:8 16 0x2a2a4075 in ReplxxLineReader::readOneLine(std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../base/common/ReplxxLineReader.cpp:112:29 17 0x2a29b499 in LineReader::readLine(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&) obj-x86_64-linux-gnu/../base/common/LineReader.cpp:81:26 18 0xb580f02 in DB::Client::mainImpl() obj-x86_64-linux-gnu/../programs/client/Client.cpp:665:33 19 0xb575825 in DB::Client::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) obj-x86_64-linux-gnu/../programs/client/Client.cpp:300:20 20 0x2a3aff25 in Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 21 0xb54c810 in mainEntryClickHouseClient(int, char**) obj-x86_64-linux-gnu/../programs/client/Client.cpp:2702:23 22 0xb326d8a in main obj-x86_64-linux-gnu/../programs/main.cpp:360:12 23 0x7ffff7dcbb24 in __libc_start_main (/usr/lib/libc.so.6+0x27b24) HINT: if you don't care about these errors you may set ASAN_OPTIONS=detect_container_overflow=0. If you suspect a false positive see also: https://github.com/google/sanitizers/wiki/AddressSanitizerContainerOverflow. SUMMARY: AddressSanitizer: container-overflow obj-x86_64-linux-gnu/../contrib/replxx/src/util.cxx:66:15 in replxx::calculate_displayed_length(char32_t const*, int) Shadow bytes around the buggy address: 0x0c408000fdc0: fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd fd 0x0c408000fdd0: fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa 0x0c408000fde0: fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa 0x0c408000fdf0: fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa 0x0c408000fe00: fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa fa =>0x0c408000fe10:[fc]fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc 0x0c408000fe20: fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc 0x0c408000fe30: fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc 0x0c408000fe40: fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc 0x0c408000fe50: fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc 0x0c408000fe60: fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc fc Shadow byte legend (one shadow byte represents 8 application bytes): Addressable: 00 Partially addressable: 01 02 03 04 05 06 07 Heap left redzone: fa Freed heap region: fd Stack left redzone: f1 Stack mid redzone: f2 Stack right redzone: f3 Stack after return: f5 Stack use after scope: f8 Global redzone: f9 Global init order: f6 Poisoned by user: f7 Container overflow: fc Array cookie: ac Intra object redzone: bb ASan internal: fe Left alloca redzone: ca Right alloca redzone: cb Shadow gap: cc ==7686==ABORTING Refs: https://github.com/ClickHouse-Extras/replxx/pull/16 v2: fix test, do not use /dev/null since it client will lock it --- contrib/replxx | 2 +- ...ient_replxx_container_overflow_long.expect | 34 +++++++++++++++++++ ...t_replxx_container_overflow_long.reference | 0 3 files changed, 35 insertions(+), 1 deletion(-) create mode 100755 tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect create mode 100644 tests/queries/0_stateless/01910_client_replxx_container_overflow_long.reference diff --git a/contrib/replxx b/contrib/replxx index 2b24f14594d..c81be6c68b1 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 2b24f14594d7606792b92544bb112a6322ba34d7 +Subproject commit c81be6c68b146f15f2096b7ef80e3f21fe27004c diff --git a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect new file mode 100755 index 00000000000..55211dca979 --- /dev/null +++ b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect @@ -0,0 +1,34 @@ +#!/usr/bin/expect -f + +log_user 0 +set timeout 60 +match_max 100000 +# A default timeout action is to do nothing, change it to fail +expect_after { + timeout { + exit 1 + } +} +set basedir [file dirname $argv0] + +# history file is not required, in-memory history is enough +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --history_file=$basedir/01910_client_replxx_container_overflow_long.history.log" +expect ":) " + +# Make a query. +send -- "SELECT 1\r" +expect "1" +expect ":) " + +# Do reverse-search. +send -- "" +expect "(reverse-i-search)" +send -- "1" +expect "(reverse-i-search)" +# This will trigger the container-overflow under ASAN before the fix. +send -- "" +expect "(reverse-i-search)" + +# Exit. +send -- "\4" +expect eof diff --git a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.reference b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.reference new file mode 100644 index 00000000000..e69de29bb2d From 57960746e2451525d07b429d8a63a5737d112db4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Jun 2021 17:12:30 +0300 Subject: [PATCH 29/43] Add getStackSize() helper --- src/Common/checkStackSize.cpp | 77 +++++++++++++++++++++-------------- 1 file changed, 47 insertions(+), 30 deletions(-) diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index 8278b510282..9f00bbed302 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -22,6 +22,52 @@ namespace DB static thread_local void * stack_address = nullptr; static thread_local size_t max_stack_size = 0; +/** + * @param address_ - stack address + * @return stack size + */ +size_t getStackSize(void **address_) +{ + using namespace DB; + + size_t size; + void *address; + +#if defined(OS_DARWIN) + // pthread_get_stacksize_np() returns a value too low for the main thread on + // OSX 10.9, http://mail.openjdk.java.net/pipermail/hotspot-dev/2013-October/011369.html + // + // Multiple workarounds possible, adopt the one made by https://github.com/robovm/robovm/issues/274 + // https://developer.apple.com/library/mac/documentation/Cocoa/Conceptual/Multithreading/CreatingThreads/CreatingThreads.html + // Stack size for the main thread is 8MB on OSX excluding the guard page size. + pthread_t thread = pthread_self(); + size = pthread_main_np() ? (8 * 1024 * 1024) : pthread_get_stacksize_np(thread); + + // stack address points to the start of the stack, not the end how it's returned by pthread_get_stackaddr_np + address = reinterpret_cast(reinterpret_cast(pthread_get_stackaddr_np(thread)) - max_stack_size); +#else + pthread_attr_t attr; +# if defined(__FreeBSD__) || defined(OS_SUNOS) + pthread_attr_init(&attr); + if (0 != pthread_attr_get_np(pthread_self(), &attr)) + throwFromErrno("Cannot pthread_attr_get_np", ErrorCodes::CANNOT_PTHREAD_ATTR); +# else + if (0 != pthread_getattr_np(pthread_self(), &attr)) + throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); +# endif + + SCOPE_EXIT({ pthread_attr_destroy(&attr); }); + + if (0 != pthread_attr_getstack(&attr, &address, &size)) + throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); +#endif // OS_DARWIN + + if (address_) + *address_ = address; + + return size; +} + /** It works fine when interpreters are instantiated by ClickHouse code in properly prepared threads, * but there are cases when ClickHouse runs as a library inside another application. * If application is using user-space lightweight threads with manually allocated stacks, @@ -34,36 +80,7 @@ __attribute__((__weak__)) void checkStackSize() using namespace DB; if (!stack_address) - { -#if defined(OS_DARWIN) - // pthread_get_stacksize_np() returns a value too low for the main thread on - // OSX 10.9, http://mail.openjdk.java.net/pipermail/hotspot-dev/2013-October/011369.html - // - // Multiple workarounds possible, adopt the one made by https://github.com/robovm/robovm/issues/274 - // https://developer.apple.com/library/mac/documentation/Cocoa/Conceptual/Multithreading/CreatingThreads/CreatingThreads.html - // Stack size for the main thread is 8MB on OSX excluding the guard page size. - pthread_t thread = pthread_self(); - max_stack_size = pthread_main_np() ? (8 * 1024 * 1024) : pthread_get_stacksize_np(thread); - - // stack_address points to the start of the stack, not the end how it's returned by pthread_get_stackaddr_np - stack_address = reinterpret_cast(reinterpret_cast(pthread_get_stackaddr_np(thread)) - max_stack_size); -#else - pthread_attr_t attr; -# if defined(__FreeBSD__) || defined(OS_SUNOS) - pthread_attr_init(&attr); - if (0 != pthread_attr_get_np(pthread_self(), &attr)) - throwFromErrno("Cannot pthread_attr_get_np", ErrorCodes::CANNOT_PTHREAD_ATTR); -# else - if (0 != pthread_getattr_np(pthread_self(), &attr)) - throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); -# endif - - SCOPE_EXIT({ pthread_attr_destroy(&attr); }); - - if (0 != pthread_attr_getstack(&attr, &stack_address, &max_stack_size)) - throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); -#endif // OS_DARWIN - } + max_stack_size = getStackSize(&stack_address); const void * frame_address = __builtin_frame_address(0); uintptr_t int_frame_address = reinterpret_cast(frame_address); From 0286673f8fcd7c3422d84273911ae4c1c1239f28 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Jun 2021 17:34:28 +0300 Subject: [PATCH 30/43] Fix __pthread_get_minstack() This is the function that should take into account TLS block, and 1MB is too high, since it will be used for sigaltstack() on SIGSEGV v0: copy-paste glibc __pthread_get_minstack() v2: return static 16K instead of 1MB --- .../glibc-compatibility/glibc-compatibility.c | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/base/glibc-compatibility/glibc-compatibility.c b/base/glibc-compatibility/glibc-compatibility.c index d4bb739a72c..1ce2659a80d 100644 --- a/base/glibc-compatibility/glibc-compatibility.c +++ b/base/glibc-compatibility/glibc-compatibility.c @@ -9,14 +9,36 @@ extern "C" { #endif #include +#include +#include +#include +/// glibc __pthread_get_minstack() will take TLS into account for the minimal +/// stack size (since you cannot use stack less then TLS block size, otherwise +/// some variables may be overwritten) +/// +/// So glibc implementation is: +/// +/// sysconf(_SC_PAGESIZE) + __static_tls_size + PTHREAD_STACK_MIN; +/// +/// But this helper cannot do this since: +/// - __pthread_get_minstack() is hidden in libc (note that rust tried to do +/// this but revert it to retry loop, for compatibility, while we cannot +/// use retry loop since this function is used for sigaltstack()) +/// - __static_tls_size is not exported in glibc +/// - it is not used anywhere except for clickhouse itself (for sigaltstack(), +/// to handle SIGSEGV only) and using PTHREAD_STACK_MIN (16k) is enough right +/// now. +/// +/// Also we cannot use getStackSize() (pthread_attr_getstack()) since it will +/// return 8MB, and this is too huge for signal stack. size_t __pthread_get_minstack(const pthread_attr_t * attr) { - return 1048576; /// This is a guess. Don't sure it is correct. + _Static_assert(PTHREAD_STACK_MIN == 16<<10, "Too small return value of __pthread_get_minstack()"); + return PTHREAD_STACK_MIN; } #include -#include #include #include From 2bc9664b52b625fbcbf1f48a28ff8a1a3f76a673 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Jun 2021 21:33:49 +0300 Subject: [PATCH 31/43] Catch "Maximum parse depth" error in fuzzer This exception should not fail the fuzzer check like here [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/25494/c6339b4c08ad8f50bf34831367de9a1ea90b1e09/fuzzer_msan/report.html#fail1 --- programs/client/Client.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 4968c7494f1..86c88b85bae 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -112,6 +112,7 @@ namespace ErrorCodes extern const int DEADLOCK_AVOIDED; extern const int UNRECOGNIZED_ARGUMENTS; extern const int SYNTAX_ERROR; + extern const int TOO_DEEP_RECURSION; } @@ -1267,7 +1268,8 @@ private: } catch (const Exception & e) { - if (e.code() != ErrorCodes::SYNTAX_ERROR) + if (e.code() != ErrorCodes::SYNTAX_ERROR && + e.code() != ErrorCodes::TOO_DEEP_RECURSION) throw; } From 9a3a1397e6d8fb15f962a7f035ba9343783acf19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Jun 2021 17:36:25 +0300 Subject: [PATCH 32/43] Fix alternative stack for SIGSEGV handling Previous stack size of 4096 wasn't enough, and this lead to that on SIGSEGV signal handler may overwrite some TLS data (like in the following example [1]) and in real binary "current_thread" was overwritten, and this leads to lack of query-id on SIGSEGV, like here [2]. [1]: https://gist.github.com/azat/2ee360fd4f2828d363b0926431afacc6 [2]: https://clickhouse-test-reports.s3.yandex.net/24411/79563953201ce6249a8fd49e22be3902ca4ee43a/fuzzer_ubsan/report.html#fail1 Fix this by using __pthread_get_minstack() that takes TLS block into account. Yes it is private, but let's try (like rust tried). This patch will also decrease TLS, since now the per-thread stack will be allocated in the heap, not on the stack of each thread. Refs: https://sourceware.org/legacy-ml/libc-alpha/2012-06/msg00335.html Refs: https://maskray.me/blog/2021-02-14-all-about-thread-local-storage Refs: https://sourceware.org/bugzilla/show_bug.cgi?id=11787 --- src/Common/ThreadStatus.cpp | 38 +++++++++++++++++++++++++++++++++---- 1 file changed, 34 insertions(+), 4 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 1493d30ea01..3bacc849eb9 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -11,6 +11,12 @@ #include +/// NOTE: clickhouse provide override for this function that will return +/// PTHREAD_STACK_MIN=16K (for compatibility reasons), +/// but this should be enough for signal stack. +extern "C" size_t __pthread_get_minstack(const pthread_attr_t * attr); + + namespace DB { @@ -25,8 +31,32 @@ thread_local ThreadStatus * current_thread = nullptr; thread_local ThreadStatus * main_thread = nullptr; #if !defined(SANITIZER) && !defined(ARCADIA_BUILD) - alignas(4096) static thread_local char alt_stack[std::max(MINSIGSTKSZ, 4096)]; - static thread_local bool has_alt_stack = false; +namespace +{ + +struct ThreadStack +{ + ThreadStack() + : size(__pthread_get_minstack(nullptr)) + , data(aligned_alloc(4096, size)) + {} + ~ThreadStack() + { + free(data); + } + + size_t getSize() const { return size; } + void* getData() const { return data; } + +private: + size_t size; + void *data; +}; + +} + +static thread_local ThreadStack alt_stack; +static thread_local bool has_alt_stack = false; #endif @@ -54,9 +84,9 @@ ThreadStatus::ThreadStatus() /// We have to call 'sigaltstack' before first 'sigaction'. (It does not work other way, for unknown reason). stack_t altstack_description{}; - altstack_description.ss_sp = alt_stack; + altstack_description.ss_sp = alt_stack.getData(); altstack_description.ss_flags = 0; - altstack_description.ss_size = sizeof(alt_stack); + altstack_description.ss_size = alt_stack.getSize(); if (0 != sigaltstack(&altstack_description, nullptr)) { From 4c391f8e9945df67b171d20612b62b45451ea8b4 Mon Sep 17 00:00:00 2001 From: Mike Kot Date: Sun, 20 Jun 2021 11:24:43 +0300 Subject: [PATCH 33/43] SYSTEM RESTORE REPLICA replica [ON CLUSTER cluster] (#13652) * initial commit: add setting and stub * typo * added test stub * fix * wip merging new integration test and code proto * adding steps interpreters * adding firstly proposed solution (moving parts etc) * added checking zookeeper path existence * fixing the include * fixing and sorting includes * fixing outdated struct * fix the name * added ast ptr as level of indirection * fix ref * updating the changes * working on test stub * fix iterator -> reference * revert rocksdb submodule update * fixed show privileges test * updated the test stub * replaced rand() with thread_local_rng(), updated the tests updated the test fixed test config path test fix removed error messages fixed the test updated the test fixed string literal fixed literal typo: = * fixed the empty replica error message * updated the test and the code with logs * updated the possible test cases, updated * added the code/test milestone comments * updated the test (added more testcases) * replaced native assert with CH one * individual replicas recursive delete fix * updated the AS db.name AST * two small logging fixes * manually generated AST fixes * Updated the test, added the possible algo change * Some thoughts about optimizing the solution: ALTER MOVE PARTITION .. TO TABLE -> move to detached/ + ALTER ... ATTACH * fix * Removed the replica sync in test as it's invalid * Some test tweaks * tmp * Rewrote the algo by using the executeQuery instead of hand-crafting the ASTPtr. Two questions still active. * tr: logging active parts * Extracted the parts moving algo into a separate helper function * Fixed the test data and the queries slightly * Replaced query to system.parts to direct invocation, started building the test that breaks on various parts. * Added the case for tables when at least one replica is alive * Updated the test to test replicas restoration by detaching/attaching * Altered the test to check restoration without replica restart * Added the tables swap in the start if the server failed last time * Hotfix when only /replicas/replica... path was deleted * Restore ZK paths while creating a replicated MergeTree table * Updated the docs, fixed the algo for individual replicas restoration case * Initial parts table storage fix, tests sync fix * Reverted individual replica restoration to general algo * Slightly optimised getDataParts * Trying another solution with parts detaching * Rewrote algo without any steps, added ON CLUSTER support * Attaching parts from other replica on restoration * Getting part checksums from ZK * Removed ON CLUSTER, finished working solution * Multiple small changes after review * Fixing parallel test * Supporting rewritten form on cluster * Test fix * Moar logging * Using source replica as checksum provider * improve test, remove some code from parser * Trying solution with move to detached + forget * Moving all parts (not only Committed) to detached * Edited docs for RESTORE REPLICA * Re-merging * minor fixes Co-authored-by: Alexander Tokmakov --- docs/en/sql-reference/statements/attach.md | 2 +- docs/en/sql-reference/statements/system.md | 52 ++++- programs/CMakeLists.txt | 1 + src/Access/AccessType.h | 1 + src/Common/ErrorCodes.cpp | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 17 +- src/Interpreters/InterpreterInsertQuery.cpp | 11 +- src/Interpreters/InterpreterSystemQuery.cpp | 113 +++++++--- src/Interpreters/InterpreterSystemQuery.h | 3 + src/Interpreters/executeDDLQueryOnCluster.cpp | 10 +- src/Parsers/ASTSystemQuery.cpp | 24 +-- src/Parsers/ASTSystemQuery.h | 2 +- src/Parsers/New/AST/SystemQuery.cpp | 2 +- src/Parsers/ParserRenameQuery.cpp | 11 +- src/Parsers/ParserSystemQuery.cpp | 80 ++++--- src/Storages/IStorage.h | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 6 + src/Storages/MergeTree/MergeTreeData.cpp | 3 +- src/Storages/MergeTree/MergeTreeData.h | 1 + .../ReplicatedMergeTreeBlockOutputStream.cpp | 6 +- .../MergeTree/registerStorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 203 ++++++++++++++---- src/Storages/StorageReplicatedMergeTree.h | 25 ++- .../test_attach_without_fetching/test.py | 7 +- .../integration/test_grant_and_revoke/test.py | 2 +- .../test_restore_replica/__init__.py | 0 .../configs/remote_servers.xml | 6 +- .../integration/test_restore_replica/test.py | 156 ++++++++++++++ .../01271_show_privileges.reference | 1 + 29 files changed, 578 insertions(+), 173 deletions(-) create mode 100644 tests/integration/test_restore_replica/__init__.py rename tests/integration/{test_attach_without_fetching => test_restore_replica}/configs/remote_servers.xml (79%) create mode 100644 tests/integration/test_restore_replica/test.py diff --git a/docs/en/sql-reference/statements/attach.md b/docs/en/sql-reference/statements/attach.md index 01783e9cb2f..bebba01980e 100644 --- a/docs/en/sql-reference/statements/attach.md +++ b/docs/en/sql-reference/statements/attach.md @@ -56,4 +56,4 @@ Result: ATTACH TABLE name UUID '' (col1 Type1, ...) ``` -It creates new table with provided structure and attaches data from table with the specified UUID. \ No newline at end of file +It creates new table with provided structure and attaches data from table with the specified UUID. diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index a17c87d2326..1708d594641 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -38,6 +38,7 @@ The list of available `SYSTEM` statements: - [START REPLICATION QUEUES](#query_language-system-start-replication-queues) - [SYNC REPLICA](#query_language-system-sync-replica) - [RESTART REPLICA](#query_language-system-restart-replica) +- [RESTORE REPLICA](#query_language-system-restore-replica) - [RESTART REPLICAS](#query_language-system-restart-replicas) ## RELOAD EMBEDDED DICTIONARIES {#query_language-system-reload-emdedded-dictionaries} @@ -290,13 +291,60 @@ After running this statement the `[db.]replicated_merge_tree_family_table_name` ### RESTART REPLICA {#query_language-system-restart-replica} -Provides possibility to reinitialize Zookeeper sessions state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of true and add tasks to Zookeeper queue if needed -Initialization replication quene based on ZooKeeper date happens in the same way as `ATTACH TABLE` statement. For a short time the table will be unavailable for any operations. +Provides possibility to reinitialize Zookeeper sessions state for `ReplicatedMergeTree` table, will compare current state with Zookeeper as source of true and add tasks to Zookeeper queue if needed. +Initialization replication queue based on ZooKeeper date happens in the same way as `ATTACH TABLE` statement. For a short time the table will be unavailable for any operations. ``` sql SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name ``` +### RESTORE REPLICA {#query_language-system-restore-replica} + +Restores a replica if data is [possibly] present but Zookeeper metadata is lost. + +Works only on readonly `ReplicatedMergeTree` tables. + +One may execute query after: + + - ZooKeeper root `/` loss. + - Replicas path `/replicas` loss. + - Individual replica path `/replicas/replica_name/` loss. + +Replica attaches locally found parts and sends info about them to Zookeeper. +Parts present on replica before metadata loss are not re-fetched from other replicas if not being outdated +(so replica restoration does not mean re-downloading all data over the network). + +Caveat: parts in all states are moved to `detached/` folder. Parts active before data loss (Committed) are attached. + +#### Syntax + +```sql +SYSTEM RESTORE REPLICA [db.]replicated_merge_tree_family_table_name [ON CLUSTER cluster_name] +``` + +Alternative syntax: + +```sql +SYSTEM RESTORE REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name +``` + +#### Example + +```sql +-- Creating table on multiple servers + +CREATE TABLE test(n UInt32) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/', '{replica}') +ORDER BY n PARTITION BY n % 10; + +INSERT INTO test SELECT * FROM numbers(1000); + +-- zookeeper_delete_path("/clickhouse/tables/test", recursive=True) <- root loss. + +SYSTEM RESTART REPLICA test; -- Table will attach as readonly as metadata is missing. +SYSTEM RESTORE REPLICA test; -- Need to execute on every replica, another way: RESTORE REPLICA test ON CLUSTER cluster +``` + ### RESTART REPLICAS {#query_language-system-restart-replicas} Provides possibility to reinitialize Zookeeper sessions state for all `ReplicatedMergeTree` tables, will compare current state with Zookeeper as source of true and add tasks to Zookeeper queue if needed diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 12aec76a303..7f85a3fc3d7 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -33,6 +33,7 @@ option (ENABLE_CLICKHOUSE_OBFUSCATOR "Table data obfuscator (convert real data t ${ENABLE_CLICKHOUSE_ALL}) # https://clickhouse.tech/docs/en/operations/utilities/odbc-bridge/ +# TODO Also needs NANODBC. if (ENABLE_ODBC) option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "HTTP-server working like a proxy to ODBC driver" ${ENABLE_CLICKHOUSE_ALL}) diff --git a/src/Access/AccessType.h b/src/Access/AccessType.h index cef2de12b30..0e295985303 100644 --- a/src/Access/AccessType.h +++ b/src/Access/AccessType.h @@ -154,6 +154,7 @@ enum class AccessType M(SYSTEM_DROP_REPLICA, "DROP REPLICA", TABLE, SYSTEM) \ M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \ M(SYSTEM_RESTART_REPLICA, "RESTART REPLICA", TABLE, SYSTEM) \ + M(SYSTEM_RESTORE_REPLICA, "RESTORE REPLICA", TABLE, SYSTEM) \ M(SYSTEM_FLUSH_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \ diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index d840830bf28..be26997d8ff 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -554,6 +554,7 @@ M(584, PROJECTION_NOT_USED) \ M(585, CANNOT_PARSE_YAML) \ M(586, CANNOT_CREATE_FILE) \ + M(587, CONCURRENT_ACCESS_NOT_SUPPORTED) \ \ M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ee208631c9b..28d88bdd8df 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -831,14 +831,17 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.attach && !create.storage && !create.columns_list) { auto database = DatabaseCatalog::instance().getDatabase(database_name); + if (database->getEngineName() == "Replicated") { auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.table); - if (typeid_cast(database.get()) && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) + + if (auto* ptr = typeid_cast(database.get()); + ptr && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) { create.database = database_name; guard->releaseTableLock(); - return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext()); + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); } } @@ -926,11 +929,13 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (need_add_to_database && database->getEngineName() == "Replicated") { auto guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table); - if (typeid_cast(database.get()) && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) + + if (auto * ptr = typeid_cast(database.get()); + ptr && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) { assertOrSetUUID(create, database); guard->releaseTableLock(); - return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext()); + return ptr->tryEnqueueReplicatedDDL(query_ptr, getContext()); } } @@ -992,8 +997,10 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } data_path = database->getTableDataPath(create); + if (!create.attach && !data_path.empty() && fs::exists(fs::path{getContext()->getPath()} / data_path)) - throw Exception(storage_already_exists_error_code, "Directory for {} data {} already exists", Poco::toLower(storage_name), String(data_path)); + throw Exception(storage_already_exists_error_code, + "Directory for {} data {} already exists", Poco::toLower(storage_name), String(data_path)); } else { diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 225bf9ec651..4d9e293d762 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -189,12 +189,11 @@ BlockIO InterpreterInsertQuery::execute() const auto & union_modes = select_query.list_of_modes; /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries - is_trivial_insert_select - = std::all_of( - union_modes.begin(), - union_modes.end(), - [](const ASTSelectWithUnionQuery::Mode & mode) { return mode == ASTSelectWithUnionQuery::Mode::ALL; }) - && std::all_of(selects.begin(), selects.end(), [](const ASTPtr & select) { return isTrivialSelect(select); }); + const auto mode_is_all = [](const auto & mode) { return mode == ASTSelectWithUnionQuery::Mode::ALL; }; + + is_trivial_insert_select = + std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) + && std::all_of(selects.begin(), selects.end(), isTrivialSelect); } if (is_trivial_insert_select) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index f97001883bd..f76d51e765b 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -43,11 +43,8 @@ # include "config_core.h" #endif - namespace DB { - - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -56,6 +53,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int TIMEOUT_EXCEEDED; extern const int TABLE_WAS_NOT_DROPPED; + extern const int NO_ZOOKEEPER; } @@ -131,6 +129,8 @@ AccessType getRequiredAccessType(StorageActionBlockType action_type) throw Exception("Unknown action type: " + std::to_string(action_type), ErrorCodes::LOGICAL_ERROR); } +constexpr std::string_view table_is_not_replicated = "Table {} is not replicated"; + } /// Implements SYSTEM [START|STOP] @@ -212,11 +212,16 @@ BlockIO InterpreterSystemQuery::execute() system_context->setSetting("profile", getContext()->getSystemProfileName()); /// Make canonical query for simpler processing - if (!query.table.empty()) + if (query.type == Type::RELOAD_DICTIONARY) + { + if (!query.database.empty()) + query.table = query.database + "." + query.table; + } + else if (!query.table.empty()) + { table_id = getContext()->resolveStorageID(StorageID(query.database, query.table), Context::ResolveOrdinary); + } - if (!query.target_dictionary.empty() && !query.database.empty()) - query.target_dictionary = query.database + "." + query.target_dictionary; volume_ptr = {}; if (!query.storage_policy.empty() && !query.volume.empty()) @@ -286,7 +291,7 @@ BlockIO InterpreterSystemQuery::execute() getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); auto & external_dictionaries_loader = system_context->getExternalDictionariesLoader(); - external_dictionaries_loader.reloadDictionary(query.target_dictionary, getContext()); + external_dictionaries_loader.reloadDictionary(query.table, getContext()); ExternalDictionariesLoader::resetAll(); @@ -296,8 +301,8 @@ BlockIO InterpreterSystemQuery::execute() { getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); executeCommandsAndThrowIfError( - [&] () { system_context->getExternalDictionariesLoader().reloadAllTriedToLoad(); }, - [&] () { system_context->getEmbeddedDictionaries().reload(); } + [&] { system_context->getExternalDictionariesLoader().reloadAllTriedToLoad(); }, + [&] { system_context->getEmbeddedDictionaries().reload(); } ); ExternalDictionariesLoader::resetAll(); break; @@ -392,8 +397,10 @@ BlockIO InterpreterSystemQuery::execute() break; case Type::RESTART_REPLICA: if (!tryRestartReplica(table_id, system_context)) - throw Exception("There is no " + query.database + "." + query.table + " replicated table", - ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs()); + break; + case Type::RESTORE_REPLICA: + restoreReplica(); break; case Type::RESTART_DISK: restartDisk(query.disk); @@ -402,14 +409,14 @@ BlockIO InterpreterSystemQuery::execute() { getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS); executeCommandsAndThrowIfError( - [&] () { if (auto query_log = getContext()->getQueryLog()) query_log->flush(true); }, - [&] () { if (auto part_log = getContext()->getPartLog("")) part_log->flush(true); }, - [&] () { if (auto query_thread_log = getContext()->getQueryThreadLog()) query_thread_log->flush(true); }, - [&] () { if (auto trace_log = getContext()->getTraceLog()) trace_log->flush(true); }, - [&] () { if (auto text_log = getContext()->getTextLog()) text_log->flush(true); }, - [&] () { if (auto metric_log = getContext()->getMetricLog()) metric_log->flush(true); }, - [&] () { if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) asynchronous_metric_log->flush(true); }, - [&] () { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); } + [&] { if (auto query_log = getContext()->getQueryLog()) query_log->flush(true); }, + [&] { if (auto part_log = getContext()->getPartLog("")) part_log->flush(true); }, + [&] { if (auto query_thread_log = getContext()->getQueryThreadLog()) query_thread_log->flush(true); }, + [&] { if (auto trace_log = getContext()->getTraceLog()) trace_log->flush(true); }, + [&] { if (auto text_log = getContext()->getTextLog()) text_log->flush(true); }, + [&] { if (auto metric_log = getContext()->getMetricLog()) metric_log->flush(true); }, + [&] { if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) asynchronous_metric_log->flush(true); }, + [&] { if (auto opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); } ); break; } @@ -423,12 +430,51 @@ BlockIO InterpreterSystemQuery::execute() return BlockIO(); } +void InterpreterSystemQuery::restoreReplica() +{ + getContext()->checkAccess(AccessType::SYSTEM_RESTORE_REPLICA, table_id); + + const zkutil::ZooKeeperPtr& zookeeper = getContext()->getZooKeeper(); + + if (zookeeper->expired()) + throw Exception(ErrorCodes::NO_ZOOKEEPER, + "Cannot restore table metadata because ZooKeeper session has expired"); + + const StoragePtr table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext()); + + auto * const table_replicated_ptr = dynamic_cast(table_ptr.get()); + + if (table_replicated_ptr == nullptr) + throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs()); + + auto & table_replicated = *table_replicated_ptr; + + StorageReplicatedMergeTree::Status status; + table_replicated.getStatus(status); + + if (!status.is_readonly) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica must be readonly"); + + const String replica_name = table_replicated.getReplicaName(); + const String& zk_root_path = status.zookeeper_path; + + if (String replica_path = zk_root_path + "replicas/" + replica_name; zookeeper->exists(replica_path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Replica path is present at {} -- nothing to restore. " + "If you are sure that metadata it lost and replica path contain some garbage, " + "then use SYSTEM DROP REPLICA query first.", replica_path); + + table_replicated.restoreMetadataInZooKeeper(); +} StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard) { getContext()->checkAccess(AccessType::SYSTEM_RESTART_REPLICA, replica); - auto table_ddl_guard = need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(replica.getDatabaseName(), replica.getTableName()) : nullptr; + auto table_ddl_guard = need_ddl_guard + ? DatabaseCatalog::instance().getDDLGuard(replica.getDatabaseName(), replica.getTableName()) + : nullptr; + auto [database, table] = DatabaseCatalog::instance().tryGetDatabaseAndTable(replica, getContext()); ASTPtr create_ast; @@ -475,28 +521,23 @@ void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context) auto & catalog = DatabaseCatalog::instance(); for (auto & elem : catalog.getDatabases()) - { - DatabasePtr & database = elem.second; - for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) - { - if (auto table = iterator->table()) - { - if (dynamic_cast(table.get())) - replica_names.emplace_back(StorageID{iterator->databaseName(), iterator->name()}); - } - } - } + for (auto it = elem.second->getTablesIterator(getContext()); it->isValid(); it->next()) + if (dynamic_cast(it->table().get())) + replica_names.emplace_back(it->databaseName(), it->name()); if (replica_names.empty()) return; TableGuards guards; + for (const auto & name : replica_names) guards.emplace(UniqueTableName{name.database_name, name.table_name}, nullptr); + for (auto & guard : guards) guard.second = catalog.getDDLGuard(guard.first.database_name, guard.first.table_name); ThreadPool pool(std::min(size_t(getNumberOfPhysicalCPUCores()), replica_names.size())); + for (auto & replica : replica_names) { LOG_TRACE(log, "Restarting replica on {}", replica.getNameForLogs()); @@ -516,7 +557,7 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query) StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (!dropReplicaImpl(query, table)) - throw Exception("Table " + table_id.getNameForLogs() + " is not replicated", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs()); } else if (!query.database.empty()) { @@ -628,7 +669,7 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery &) LOG_TRACE(log, "SYNC REPLICA {}: OK", table_id.getNameForLogs()); } else - throw Exception("Table " + table_id.getNameForLogs() + " is not replicated", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs()); } void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &) @@ -659,6 +700,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() const auto & query = query_ptr->as(); using Type = ASTSystemQuery::Type; AccessRightsElements required_access; + switch (query.type) { case Type::SHUTDOWN: [[fallthrough]]; @@ -770,6 +812,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() required_access.emplace_back(AccessType::SYSTEM_DROP_REPLICA, query.database, query.table); break; } + case Type::RESTORE_REPLICA: + { + required_access.emplace_back(AccessType::SYSTEM_RESTORE_REPLICA, query.database, query.table); + break; + } case Type::SYNC_REPLICA: { required_access.emplace_back(AccessType::SYSTEM_SYNC_REPLICA, query.database, query.table); diff --git a/src/Interpreters/InterpreterSystemQuery.h b/src/Interpreters/InterpreterSystemQuery.h index 297f7225a92..6d1ad78a991 100644 --- a/src/Interpreters/InterpreterSystemQuery.h +++ b/src/Interpreters/InterpreterSystemQuery.h @@ -49,6 +49,9 @@ private: void restartReplicas(ContextMutablePtr system_context); void syncReplica(ASTSystemQuery & query); + + void restoreReplica(); + void dropReplica(ASTSystemQuery & query); bool dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table); void flushDistributed(ASTSystemQuery & query); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 8a6abf7714f..c5dec2cf214 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -102,12 +102,10 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, /// The current database in a distributed query need to be replaced with either /// the local current database or a shard's default database. - bool need_replace_current_database - = (std::find_if( - query_requires_access.begin(), - query_requires_access.end(), - [](const AccessRightsElement & elem) { return elem.isEmptyDatabase(); }) - != query_requires_access.end()); + bool need_replace_current_database = std::any_of( + query_requires_access.begin(), + query_requires_access.end(), + [](const AccessRightsElement & elem) { return elem.isEmptyDatabase(); }); bool use_local_default_database = false; const String & current_database = context->getCurrentDatabase(); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index bf94df0bf50..5d01e124b0e 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -44,6 +44,8 @@ const char * ASTSystemQuery::typeToString(Type type) return "RESTART REPLICAS"; case Type::RESTART_REPLICA: return "RESTART REPLICA"; + case Type::RESTORE_REPLICA: + return "RESTORE REPLICA"; case Type::DROP_REPLICA: return "DROP REPLICA"; case Type::SYNC_REPLICA: @@ -119,18 +121,6 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, << (settings.hilite ? hilite_none : ""); }; - auto print_database_dictionary = [&] - { - settings.ostr << " "; - if (!database.empty()) - { - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(database) - << (settings.hilite ? hilite_none : "") << "."; - } - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(target_dictionary) - << (settings.hilite ? hilite_none : ""); - }; - auto print_drop_replica = [&] { settings.ostr << " " << quoteString(replica); @@ -187,14 +177,14 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, else if (!volume.empty()) print_on_volume(); } - else if (type == Type::RESTART_REPLICA || type == Type::SYNC_REPLICA || type == Type::FLUSH_DISTRIBUTED) + else if ( type == Type::RESTART_REPLICA + || type == Type::RESTORE_REPLICA + || type == Type::SYNC_REPLICA + || type == Type::FLUSH_DISTRIBUTED + || type == Type::RELOAD_DICTIONARY) { print_database_table(); } - else if (type == Type::RELOAD_DICTIONARY) - { - print_database_dictionary(); - } else if (type == Type::DROP_REPLICA) { print_drop_replica(); diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index 6cd1443155f..cbe82cd936f 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -32,6 +32,7 @@ public: START_LISTEN_QUERIES, RESTART_REPLICAS, RESTART_REPLICA, + RESTORE_REPLICA, DROP_REPLICA, SYNC_REPLICA, RELOAD_DICTIONARY, @@ -65,7 +66,6 @@ public: Type type = Type::UNKNOWN; - String target_dictionary; String target_model; String database; String table; diff --git a/src/Parsers/New/AST/SystemQuery.cpp b/src/Parsers/New/AST/SystemQuery.cpp index 2be9ff951e0..d2fda6a3fbc 100644 --- a/src/Parsers/New/AST/SystemQuery.cpp +++ b/src/Parsers/New/AST/SystemQuery.cpp @@ -133,7 +133,7 @@ ASTPtr SystemQuery::convertToOld() const { auto table = std::static_pointer_cast(get(TABLE)->convertToOld()); query->database = table->getDatabaseName(); - query->target_dictionary = table->shortName(); + query->table = table->getTableId().table_name; } break; case QueryType::REPLICATED_SENDS: diff --git a/src/Parsers/ParserRenameQuery.cpp b/src/Parsers/ParserRenameQuery.cpp index e3b35249cd6..c42a0af88b2 100644 --- a/src/Parsers/ParserRenameQuery.cpp +++ b/src/Parsers/ParserRenameQuery.cpp @@ -95,21 +95,18 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTRenameQuery::Elements elements; - auto ignore_delim = [&]() - { - return exchange ? s_and.ignore(pos) : s_to.ignore(pos); - }; + const auto ignore_delim = [&] { return exchange ? s_and.ignore(pos) : s_to.ignore(pos); }; while (true) { if (!elements.empty() && !s_comma.ignore(pos)) break; - elements.push_back(ASTRenameQuery::Element()); + ASTRenameQuery::Element& ref = elements.emplace_back(); - if (!parseDatabaseAndTable(elements.back().from, pos, expected) + if (!parseDatabaseAndTable(ref.from, pos, expected) || !ignore_delim() - || !parseDatabaseAndTable(elements.back().to, pos, expected)) + || !parseDatabaseAndTable(ref.to, pos, expected)) return false; } diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index a1487468ab3..66bd39e0202 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -15,6 +15,47 @@ namespace ErrorCodes namespace DB { +static bool parseQueryWithOnClusterAndMaybeTable(std::shared_ptr & res, IParser::Pos & pos, + Expected & expected, bool require_table, bool allow_string_literal) +{ + /// Better form for user: SYSTEM table ON CLUSTER cluster + /// Query rewritten form + form while executing on cluster: SYSTEM ON CLUSTER cluster table + /// Need to support both + String cluster; + bool parsed_on_cluster = false; + + if (ParserKeyword{"ON"}.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster, expected)) + return false; + parsed_on_cluster = true; + } + + bool parsed_table = false; + if (allow_string_literal) + { + ASTPtr ast; + if (ParserStringLiteral{}.parse(pos, ast, expected)) + { + res->database = {}; + res->table = ast->as().value.safeGet(); + parsed_table = true; + } + } + + if (!parsed_table) + parsed_table = parseDatabaseAndTableName(pos, expected, res->database, res->table); + + if (!parsed_table && require_table) + return false; + + if (!parsed_on_cluster && ParserKeyword{"ON"}.ignore(pos, expected)) + if (!ASTQueryWithOnCluster::parse(pos, cluster, expected)) + return false; + + res->cluster = cluster; + return true; +} bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { @@ -43,17 +84,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & { case Type::RELOAD_DICTIONARY: { - String cluster_str; - if (ParserKeyword{"ON"}.ignore(pos, expected)) - { - if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) - return false; - } - res->cluster = cluster_str; - ASTPtr ast; - if (ParserStringLiteral{}.parse(pos, ast, expected)) - res->target_dictionary = ast->as().value.safeGet(); - else if (!parseDatabaseAndTableName(pos, expected, res->database, res->target_dictionary)) + if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ true, /* allow_string_literal = */ true)) return false; break; } @@ -145,24 +176,21 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & break; } + /// FLUSH DISTRIBUTED requires table + /// START/STOP DISTRIBUTED SENDS does not require table case Type::STOP_DISTRIBUTED_SENDS: case Type::START_DISTRIBUTED_SENDS: - case Type::FLUSH_DISTRIBUTED: { - String cluster_str; - if (ParserKeyword{"ON"}.ignore(pos, expected)) - { - if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) - return false; - } - res->cluster = cluster_str; - if (!parseDatabaseAndTableName(pos, expected, res->database, res->table)) - { - /// FLUSH DISTRIBUTED requires table - /// START/STOP DISTRIBUTED SENDS does not require table - if (res->type == Type::FLUSH_DISTRIBUTED) - return false; - } + if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ false, /* allow_string_literal = */ false)) + return false; + break; + } + + case Type::FLUSH_DISTRIBUTED: + case Type::RESTORE_REPLICA: + { + if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ true, /* allow_string_literal = */ false)) + return false; break; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index a96748ceb63..0e8c7e0a263 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -190,9 +190,10 @@ protected: /// Initially reserved virtual column name may be shadowed by real column. bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const; - private: + StorageID storage_id; + mutable std::mutex id_mutex; /// Multiversion storage metadata. Allows to read/write storage metadata diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f8ff7fe697a..54fcfc1adc9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -223,6 +223,12 @@ public: DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor }; + static constexpr auto all_part_states = + { + State::Temporary, State::PreCommitted, State::Committed, State::Outdated, State::Deleting, + State::DeleteOnDestroy + }; + using TTLInfo = MergeTreeDataPartTTLInfo; using TTLInfos = MergeTreeDataPartTTLInfos; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b0bf0c8e672..abc37f52ff9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2156,8 +2156,7 @@ bool MergeTreeData::renameTempPartAndReplace( LOG_TRACE(log, "Renaming temporary part {} to {}.", part->relative_path, part_name); - auto it_duplicate = data_parts_by_info.find(part_info); - if (it_duplicate != data_parts_by_info.end()) + if (auto it_duplicate = data_parts_by_info.find(part_info); it_duplicate != data_parts_by_info.end()) { String message = "Part " + (*it_duplicate)->getNameWithState() + " already exists"; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4f33aa30bdc..65d875aa9cf 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -402,6 +402,7 @@ public: /// Returns a copy of the list so that the caller shouldn't worry about locks. DataParts getDataParts(const DataPartStates & affordable_states) const; + /// Returns sorted list of the parts with specified states /// out_states will contain snapshot of each part state DataPartsVector getDataPartsVector( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 7f167f929e5..4a73658e8a4 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -262,8 +262,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( { log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART; - /// We don't need to involve ZooKeeper to obtain the checksums as by the time we get - /// the MutableDataPartPtr here, we already have the data thus being able to + /// We don't need to involve ZooKeeper to obtain checksums as by the time we get + /// MutableDataPartPtr here, we already have the data thus being able to /// calculate the checksums. log_entry.part_checksum = part->checksums.getTotalChecksumHex(); } @@ -384,6 +384,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( MergeTreeData::Transaction transaction(storage); /// If you can not add a part to ZK, we'll remove it back from the working set. bool renamed = false; + try { renamed = storage.renameTempPartAndAdd(part, nullptr, &transaction); @@ -394,6 +395,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) throw; } + if (!renamed) { if (is_already_existing_part) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index ce7fc38b904..b3b9ce31ff5 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -257,7 +257,7 @@ If you use the Replicated version of engines, see https://clickhouse.tech/docs/e static StoragePtr create(const StorageFactory::Arguments & args) { - /** [Replicated][|Summing|Collapsing|Aggregating|Replacing|Graphite]MergeTree (2 * 7 combinations) engines + /** [Replicated][|Summing|VersionedCollapsing|Collapsing|Aggregating|Replacing|Graphite]MergeTree (2 * 7 combinations) engines * The argument for the engine should be: * - (for Replicated) The path to the table in ZooKeeper * - (for Replicated) Replica name in ZooKeeper diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c11f1580a2e..47f6bbd0ccc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1,5 +1,6 @@ #include +#include "Common/hex.h" #include #include #include @@ -63,10 +64,13 @@ #include #include +#include #include +#include +#include +#include #include #include -#include #include @@ -135,6 +139,7 @@ namespace ErrorCodes extern const int INTERSERVER_SCHEME_DOESNT_MATCH; extern const int DUPLICATE_DATA_PART; extern const int BAD_ARGUMENTS; + extern const int CONCURRENT_ACCESS_NOT_SUPPORTED; } namespace ActionLocks @@ -153,10 +158,6 @@ static const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000; static const auto MUTATIONS_FINALIZING_SLEEP_MS = 1 * 1000; static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000; - -std::atomic_uint StorageReplicatedMergeTree::total_fetches {0}; - - void StorageReplicatedMergeTree::setZooKeeper() { /// Every ReplicatedMergeTree table is using only one ZooKeeper session. @@ -376,7 +377,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( if (attach && !current_zookeeper->exists(zookeeper_path + "/metadata")) { - LOG_WARNING(log, "No metadata in ZooKeeper: table will be in readonly mode."); + LOG_WARNING(log, "No metadata in ZooKeeper for {}: table will be in readonly mode.", zookeeper_path); is_readonly = true; has_metadata_in_zookeeper = false; return; @@ -384,10 +385,20 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( auto metadata_snapshot = getInMemoryMetadataPtr(); + /// May it be ZK lost not the whole root, so the upper check passed, but only the /replicas/replica + /// folder. + if (attach && !current_zookeeper->exists(replica_path)) + { + LOG_WARNING(log, "No metadata in ZooKeeper for {}: table will be in readonly mode", replica_path); + is_readonly = true; + has_metadata_in_zookeeper = false; + return; + } + if (!attach) { if (!getDataParts().empty()) - throw Exception("Data directory for table already containing data parts" + throw Exception("Data directory for table already contains data parts" " - probably it was unclean DROP table or manual intervention." " You must either clear directory by hand or use ATTACH TABLE" " instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA); @@ -433,13 +444,17 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( { /// In old tables this node may missing or be empty String replica_metadata; - bool replica_metadata_exists = current_zookeeper->tryGet(replica_path + "/metadata", replica_metadata); + const bool replica_metadata_exists = current_zookeeper->tryGet(replica_path + "/metadata", replica_metadata); + if (!replica_metadata_exists || replica_metadata.empty()) { /// We have to check shared node granularity before we create ours. other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper(); + ReplicatedMergeTreeTableMetadata current_metadata(*this, metadata_snapshot); - current_zookeeper->createOrUpdate(replica_path + "/metadata", current_metadata.toString(), zkutil::CreateMode::Persistent); + + current_zookeeper->createOrUpdate(replica_path + "/metadata", current_metadata.toString(), + zkutil::CreateMode::Persistent); } checkTableStructure(replica_path, metadata_snapshot); @@ -460,8 +475,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); metadata_version = metadata_stat.version; } - /// Temporary directories contain untinalized results of Merges or Fetches (after forced restart) - /// and don't allow to reinitialize them, so delete each of them immediately + /// Temporary directories contain uninitialized results of Merges or Fetches (after forced restart), + /// don't allow to reinitialize them, delete each of them immediately. clearOldTemporaryDirectories(0); clearOldWriteAheadLogs(); } @@ -727,12 +742,13 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada String replicas_value; if (!zookeeper->tryGet(zookeeper_path + "/replicas", replicas_value, &replicas_stat)) - throw Exception(fmt::format("Cannot create a replica of the table {}, because the last replica of the table was dropped right now", - zookeeper_path), ErrorCodes::ALL_REPLICAS_LOST); + throw Exception(ErrorCodes::ALL_REPLICAS_LOST, + "Cannot create a replica of the table {}, because the last replica of the table was dropped right now", + zookeeper_path); /// It is not the first replica, we will mark it as "lost", to immediately repair (clone) from existing replica. /// By the way, it's possible that the replica will be first, if all previous replicas were removed concurrently. - String is_lost_value = replicas_stat.numChildren ? "1" : "0"; + const String is_lost_value = replicas_stat.numChildren ? "1" : "0"; Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", @@ -761,21 +777,18 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada Coordination::Responses responses; code = zookeeper->tryMulti(ops, responses); - if (code == Coordination::Error::ZNODEEXISTS) + + switch (code) { - throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST); - } - else if (code == Coordination::Error::ZBADVERSION) - { - LOG_ERROR(log, "Retrying createReplica(), because some other replicas were created at the same time"); - } - else if (code == Coordination::Error::ZNONODE) - { - throw Exception("Table " + zookeeper_path + " was suddenly removed.", ErrorCodes::ALL_REPLICAS_LOST); - } - else - { - zkutil::KeeperMultiException::check(code, ops, responses); + case Coordination::Error::ZNODEEXISTS: + throw Exception(ErrorCodes::REPLICA_IS_ALREADY_EXIST, "Replica {} already exists", replica_path); + case Coordination::Error::ZBADVERSION: + LOG_ERROR(log, "Retrying createReplica(), because some other replicas were created at the same time"); + break; + case Coordination::Error::ZNONODE: + throw Exception(ErrorCodes::ALL_REPLICAS_LOST, "Table {} was suddenly removed", zookeeper_path); + default: + zkutil::KeeperMultiException::check(code, ops, responses); } } while (code == Coordination::Error::ZBADVERSION); } @@ -1123,6 +1136,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) size_t unexpected_parts_nonnew = 0; UInt64 unexpected_parts_nonnew_rows = 0; UInt64 unexpected_parts_rows = 0; + for (const auto & part : unexpected_parts) { if (part->info.level > 0) @@ -1134,20 +1148,17 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) unexpected_parts_rows += part->rows_count; } - /// Additional helpful statistics - auto get_blocks_count_in_data_part = [&] (const String & part_name) -> UInt64 - { - MergeTreePartInfo part_info; - if (MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version)) - return part_info.getBlocksCount(); + const UInt64 parts_to_fetch_blocks = std::accumulate(parts_to_fetch.cbegin(), parts_to_fetch.cend(), 0, + [&](UInt64 acc, const String& part_name) + { + MergeTreePartInfo part_info; - LOG_ERROR(log, "Unexpected part name: {}", part_name); - return 0; - }; + if (MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version)) + return acc + part_info.getBlocksCount(); - UInt64 parts_to_fetch_blocks = 0; - for (const String & name : parts_to_fetch) - parts_to_fetch_blocks += get_blocks_count_in_data_part(name); + LOG_ERROR(log, "Unexpected part name: {}", part_name); + return acc; + }); /** We can automatically synchronize data, * if the ratio of the total number of errors to the total number of parts (minimum - on the local filesystem or in ZK) @@ -1499,7 +1510,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) { if (MutableDataPartPtr part = attachPartHelperFoundValidPart(entry); part) { - LOG_TRACE(log, "Found valid part to attach from local data, preparing the transaction"); + LOG_TRACE(log, "Found valid local part for {}, preparing the transaction", part->name); Transaction transaction(*this); @@ -1512,7 +1523,9 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) return true; } - LOG_TRACE(log, "Didn't find part with the correct checksums, will fetch it from other replica"); + LOG_TRACE(log, "Didn't find valid local part for {} ({}), will fetch it from other replica", + entry.new_part_name, + entry.actual_new_part_name); } if (is_get_or_attach && entry.source_replica == replica_name) @@ -2732,6 +2745,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo /// Remove local parts if source replica does not have them, because such parts will never be fetched by other replicas. Strings local_parts_in_zk = zookeeper->getChildren(fs::path(replica_path) / "parts"); Strings parts_to_remove_from_zk; + for (const auto & part : local_parts_in_zk) { if (active_parts_set.getContainingPart(part).empty()) @@ -2740,10 +2754,13 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo LOG_WARNING(log, "Source replica does not have part {}. Removing it from ZooKeeper.", part); } } + tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_from_zk); auto local_active_parts = getDataParts(); + DataPartsVector parts_to_remove_from_working_set; + for (const auto & part : local_active_parts) { if (active_parts_set.getContainingPart(part->name).empty()) @@ -2756,6 +2773,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo if (getSettings()->detach_old_local_parts_when_cloning_replica) { auto metadata_snapshot = getInMemoryMetadataPtr(); + for (const auto & part : parts_to_remove_from_working_set) { LOG_INFO(log, "Detaching {}", part->relative_path); @@ -2768,7 +2786,35 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const String & name : active_parts) { LogEntry log_entry; - log_entry.type = LogEntry::GET_PART; + + if (!are_restoring_replica) + log_entry.type = LogEntry::GET_PART; + else + { + LOG_DEBUG(log, "Obtaining checksum for path {}", name); + + // The part we want to fetch is probably present in detached/ folder. + // However, we need to get part's checksum to check if it's not corrupt. + log_entry.type = LogEntry::ATTACH_PART; + + MinimalisticDataPartChecksums desired_checksums; + + const fs::path part_path = fs::path(source_path) / "parts" / name; + + const String part_znode = zookeeper->get(part_path); + + if (!part_znode.empty()) + desired_checksums = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums(); + else + { + String desired_checksums_str = zookeeper->get(part_path / "checksums"); + desired_checksums = MinimalisticDataPartChecksums::deserializeFrom(desired_checksums_str); + } + + const auto [lo, hi] = desired_checksums.hash_of_all_files; + log_entry.part_checksum = getHexUIntUppercase(hi) + getHexUIntUppercase(lo); + } + log_entry.source_replica = ""; log_entry.new_part_name = name; log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name); @@ -2868,6 +2914,7 @@ void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zooke Coordination::Stat is_lost_stat; bool is_new_replica = true; String res; + if (zookeeper->tryGet(fs::path(replica_path) / "is_lost", res, &is_lost_stat)) { if (res == "0") @@ -3968,6 +4015,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora MinimalisticDataPartChecksums desired_checksums; String part_path = fs::path(source_replica_path) / "parts" / part_name; String part_znode = zookeeper->get(part_path); + if (!part_znode.empty()) desired_checksums = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums(); else @@ -5030,6 +5078,59 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St return true; } +void StorageReplicatedMergeTree::restoreMetadataInZooKeeper() +{ + LOG_INFO(log, "Restoring replica metadata"); + + if (!is_readonly || has_metadata_in_zookeeper) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: replica is not readonly"); + + if (are_restoring_replica.exchange(true)) + throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Replica restoration in progress"); + + auto metadata_snapshot = getInMemoryMetadataPtr(); + + const DataPartsVector all_parts = getDataPartsVector(IMergeTreeDataPart::all_part_states); + Strings active_parts_names; + + /// Why all parts (not only Committed) are moved to detached/: + /// After ZK metadata restoration ZK resets sequential counters (including block number counters), so one may + /// potentially encounter a situation that a part we want to attach already exists. + for (const auto & part : all_parts) + { + if (part->getState() == DataPartState::Committed) + active_parts_names.push_back(part->name); + + forgetPartAndMoveToDetached(part); + } + + LOG_INFO(log, "Moved all parts to detached/"); + + const bool is_first_replica = createTableIfNotExists(metadata_snapshot); + + LOG_INFO(log, "Created initial ZK nodes, replica is first: {}", is_first_replica); + + if (!is_first_replica) + createReplica(metadata_snapshot); + + createNewZooKeeperNodes(); + + LOG_INFO(log, "Created ZK nodes for table"); + + is_readonly = false; + has_metadata_in_zookeeper = true; + + if (is_first_replica) + for (const String& part_name : active_parts_names) + attachPartition(std::make_shared(part_name), metadata_snapshot, true, getContext()); + + LOG_INFO(log, "Attached all partitions, starting table"); + + startup(); + + are_restoring_replica.store(false); +} + void StorageReplicatedMergeTree::dropPartNoWaitNoThrow(const String & part_name) { assertNotReadonly(); @@ -6938,8 +7039,10 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition( zookeeper.get(alter_partition_version_path, &alter_partition_version_stat); MergeTreePartInfo drop_range_info; - /// It prevent other replicas from assigning merges which intersect locked block number. + + /// It would prevent other replicas from assigning merges which intersect locked block number. std::optional delimiting_block_lock; + if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info, delimiting_block_lock)) { LOG_INFO(log, "Will not drop partition {}, it is empty.", partition_id); @@ -6960,23 +7063,31 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition( entry.create_time = time(nullptr); Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential)); + + ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry.toString(), + zkutil::CreateMode::PersistentSequential)); + /// Check and update version to avoid race with REPLACE_RANGE. /// Otherwise new parts covered by drop_range_info may appear after execution of current DROP_RANGE entry /// as a result of execution of concurrently created REPLACE_RANGE entry. ops.emplace_back(zkutil::makeCheckRequest(alter_partition_version_path, alter_partition_version_stat.version)); ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", -1)); + /// Just update version, because merges assignment relies on it ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); delimiting_block_lock->getUnlockOps(ops); + if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(ops); + Coordination::Responses responses; Coordination::Error code = zookeeper.tryMulti(ops, responses); + if (code == Coordination::Error::ZOK) delimiting_block_lock->assumeUnlocked(); else if (code == Coordination::Error::ZBADVERSION) - throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, "Cannot assign ALTER PARTITION, because another ALTER PARTITION query was concurrently executed"); + throw Exception(ErrorCodes::CANNOT_ASSIGN_ALTER, + "Cannot assign ALTER PARTITION because another ALTER PARTITION query was concurrently executed"); else zkutil::KeeperMultiException::check(code, ops, responses); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 505eb4e87c4..396ec7a1741 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -35,7 +35,7 @@ namespace DB { -/** The engine that uses the merge tree (see MergeTreeData) and replicated through ZooKeeper. +/** The engine that uses the merge tree (see MergeTreeData) and is replicated through ZooKeeper. * * ZooKeeper is used for the following things: * - the structure of the table (/metadata, /columns) @@ -57,6 +57,7 @@ namespace DB * Log - a sequence of entries (LogEntry) about what to do. * Each entry is one of: * - normal data insertion (GET), + * - data insertion with a possible attach from local data (ATTACH), * - merge (MERGE), * - delete the partition (DROP). * @@ -65,10 +66,8 @@ namespace DB * Despite the name of the "queue", execution can be reordered, if necessary (shouldExecuteLogEntry, executeLogEntry). * In addition, the records in the queue can be generated independently (not from the log), in the following cases: * - when creating a new replica, actions are put on GET from other replicas (createReplica); - * - if the part is corrupt (removePartAndEnqueueFetch) or absent during the check (at start - checkParts, while running - searchForMissingPart), - * actions are put on GET from other replicas; - * - * TODO Update the GET part after rewriting the code (search locally). + * - if the part is corrupt (removePartAndEnqueueFetch) or absent during the check + * (at start - checkParts, while running - searchForMissingPart), actions are put on GET from other replicas; * * The replica to which INSERT was made in the queue will also have an entry of the GET of this data. * Such an entry is considered to be executed as soon as the queue handler sees it. @@ -240,6 +239,13 @@ public: /// Get best replica having this partition on S3 String getSharedDataReplica(const IMergeTreeDataPart & part) const; + inline String getReplicaName() const { return replica_name; } + + /// Restores table metadata if ZooKeeper lost it. + /// Used only on restarted readonly replicas (not checked). All active (Committed) parts are moved to detached/ + /// folder and attached. Parts in all other states are just moved to detached/ folder. + void restoreMetadataInZooKeeper(); + /// Get throttler for replicated fetches ThrottlerPtr getFetchesThrottler() const { @@ -253,6 +259,8 @@ public: } private: + std::atomic_bool are_restoring_replica {false}; + /// Get a sequential consistent view of current parts. ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; @@ -332,7 +340,7 @@ private: Poco::Event partial_shutdown_event {false}; /// Poco::Event::EVENT_MANUALRESET /// Limiting parallel fetches per node - static std::atomic_uint total_fetches; + static inline std::atomic_uint total_fetches {0}; /// Limiting parallel fetches per one table std::atomic_uint current_table_fetches {0}; @@ -389,8 +397,9 @@ private: */ bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); - /** Creates a replica in ZooKeeper and adds to the queue all that it takes to catch up with the rest of the replicas. - */ + /** + * Creates a replica in ZooKeeper and adds to the queue all that it takes to catch up with the rest of the replicas. + */ void createReplica(const StorageMetadataPtr & metadata_snapshot); /** Create nodes in the ZK, which must always be, but which might not exist when older versions of the server are running. diff --git a/tests/integration/test_attach_without_fetching/test.py b/tests/integration/test_attach_without_fetching/test.py index 605ca6a4f51..874f5b36ddc 100644 --- a/tests/integration/test_attach_without_fetching/test.py +++ b/tests/integration/test_attach_without_fetching/test.py @@ -16,11 +16,10 @@ def fill_node(node): '''.format(replica=node.name)) cluster = ClickHouseCluster(__file__) -configs =["configs/remote_servers.xml"] -node_1 = cluster.add_instance('replica1', with_zookeeper=True, main_configs=configs) -node_2 = cluster.add_instance('replica2', with_zookeeper=True, main_configs=configs) -node_3 = cluster.add_instance('replica3', with_zookeeper=True, main_configs=configs) +node_1 = cluster.add_instance('replica1', with_zookeeper=True) +node_2 = cluster.add_instance('replica2', with_zookeeper=True) +node_3 = cluster.add_instance('replica3', with_zookeeper=True) @pytest.fixture(scope="module") def start_cluster(): diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index c1be16fe17d..1124f072a06 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -151,7 +151,7 @@ def test_grant_all_on_table(): instance.query("GRANT ALL ON test.table TO A WITH GRANT OPTION") instance.query("GRANT ALL ON test.table TO B", user='A') assert instance.query( - "SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" + "SHOW GRANTS FOR B") == "GRANT SHOW TABLES, SHOW COLUMNS, SHOW DICTIONARIES, SELECT, INSERT, ALTER, CREATE TABLE, CREATE VIEW, CREATE DICTIONARY, DROP TABLE, DROP VIEW, DROP DICTIONARY, TRUNCATE, OPTIMIZE, SYSTEM MERGES, SYSTEM TTL MERGES, SYSTEM FETCHES, SYSTEM MOVES, SYSTEM SENDS, SYSTEM REPLICATION QUEUES, SYSTEM DROP REPLICA, SYSTEM SYNC REPLICA, SYSTEM RESTART REPLICA, SYSTEM RESTORE REPLICA, SYSTEM FLUSH DISTRIBUTED, dictGet ON test.table TO B\n" instance.query("REVOKE ALL ON test.table FROM B", user='A') assert instance.query("SHOW GRANTS FOR B") == "" diff --git a/tests/integration/test_restore_replica/__init__.py b/tests/integration/test_restore_replica/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_attach_without_fetching/configs/remote_servers.xml b/tests/integration/test_restore_replica/configs/remote_servers.xml similarity index 79% rename from tests/integration/test_attach_without_fetching/configs/remote_servers.xml rename to tests/integration/test_restore_replica/configs/remote_servers.xml index 7978f921b2e..0709f97551a 100644 --- a/tests/integration/test_attach_without_fetching/configs/remote_servers.xml +++ b/tests/integration/test_restore_replica/configs/remote_servers.xml @@ -4,15 +4,15 @@ true - node_1_1 + replica1 9000 - node_1_2 + replica2 9000 - node_1_3 + replica3 9000 diff --git a/tests/integration/test_restore_replica/test.py b/tests/integration/test_restore_replica/test.py new file mode 100644 index 00000000000..4197c064243 --- /dev/null +++ b/tests/integration/test_restore_replica/test.py @@ -0,0 +1,156 @@ +import time +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseKiller +from helpers.test_tools import assert_eq_with_retry +from helpers.network import PartitionManager + +def fill_nodes(nodes): + for node in nodes: + node.query( + ''' + CREATE TABLE test(n UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/', '{replica}') + ORDER BY n PARTITION BY n % 10; + '''.format(replica=node.name)) + +cluster = ClickHouseCluster(__file__) +configs =["configs/remote_servers.xml"] + +node_1 = cluster.add_instance('replica1', with_zookeeper=True, main_configs=configs) +node_2 = cluster.add_instance('replica2', with_zookeeper=True, main_configs=configs) +node_3 = cluster.add_instance('replica3', with_zookeeper=True, main_configs=configs) +nodes = [node_1, node_2, node_3] + +def fill_table(): + node_1.query("TRUNCATE TABLE test") + + for node in nodes: + node.query("SYSTEM SYNC REPLICA test") + + check_data(0, 0) + + # it will create multiple parts in each partition and probably cause merges + node_1.query("INSERT INTO test SELECT number + 0 FROM numbers(200)") + node_1.query("INSERT INTO test SELECT number + 200 FROM numbers(200)") + node_1.query("INSERT INTO test SELECT number + 400 FROM numbers(200)") + node_1.query("INSERT INTO test SELECT number + 600 FROM numbers(200)") + node_1.query("INSERT INTO test SELECT number + 800 FROM numbers(200)") + check_data(499500, 1000) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + fill_nodes(nodes) + yield cluster + + except Exception as ex: + print(ex) + + finally: + cluster.shutdown() + +def check_data(_sum: int, count: int) -> None: + res = "{}\t{}\n".format(_sum, count) + assert_eq_with_retry(node_1, "SELECT sum(n), count() FROM test", res) + assert_eq_with_retry(node_2, "SELECT sum(n), count() FROM test", res) + assert_eq_with_retry(node_3, "SELECT sum(n), count() FROM test", res) + +def check_after_restoration(): + check_data(1999000, 2000) + + for node in nodes: + node.query_and_get_error("SYSTEM RESTORE REPLICA test") + +def test_restore_replica_invalid_tables(start_cluster): + print("Checking the invocation on non-existent and non-replicated tables") + node_1.query_and_get_error("SYSTEM RESTORE REPLICA i_dont_exist_42") + node_1.query_and_get_error("SYSTEM RESTORE REPLICA no_db.i_dont_exist_42") + node_1.query_and_get_error("SYSTEM RESTORE REPLICA system.numbers") + +def test_restore_replica_sequential(start_cluster): + zk = cluster.get_kazoo_client('zoo1') + fill_table() + + print("Deleting root ZK path metadata") + zk.delete("/clickhouse/tables/test", recursive=True) + assert zk.exists("/clickhouse/tables/test") is None + + node_1.query("SYSTEM RESTART REPLICA test") + node_1.query_and_get_error("INSERT INTO test SELECT number AS num FROM numbers(1000,2000) WHERE num % 2 = 0") + + print("Restoring replica1") + + node_1.query("SYSTEM RESTORE REPLICA test") + assert zk.exists("/clickhouse/tables/test") + check_data(499500, 1000) + + node_1.query("INSERT INTO test SELECT number + 1000 FROM numbers(1000)") + + print("Restoring other replicas") + + node_2.query("SYSTEM RESTART REPLICA test") + node_2.query("SYSTEM RESTORE REPLICA test") + + node_3.query("SYSTEM RESTART REPLICA test") + node_3.query("SYSTEM RESTORE REPLICA test") + + node_2.query("SYSTEM SYNC REPLICA test") + node_3.query("SYSTEM SYNC REPLICA test") + + check_after_restoration() + +def test_restore_replica_parallel(start_cluster): + zk = cluster.get_kazoo_client('zoo1') + fill_table() + + print("Deleting root ZK path metadata") + zk.delete("/clickhouse/tables/test", recursive=True) + assert zk.exists("/clickhouse/tables/test") is None + + node_1.query("SYSTEM RESTART REPLICA test") + node_1.query_and_get_error("INSERT INTO test SELECT number AS num FROM numbers(1000,2000) WHERE num % 2 = 0") + + print("Restoring replicas in parallel") + + node_2.query("SYSTEM RESTART REPLICA test") + node_3.query("SYSTEM RESTART REPLICA test") + + node_1.query("SYSTEM RESTORE REPLICA test ON CLUSTER test_cluster") + + assert zk.exists("/clickhouse/tables/test") + check_data(499500, 1000) + + node_1.query("INSERT INTO test SELECT number + 1000 FROM numbers(1000)") + + check_after_restoration() + +def test_restore_replica_alive_replicas(start_cluster): + zk = cluster.get_kazoo_client('zoo1') + fill_table() + + print("Deleting replica2 path, trying to restore replica1") + zk.delete("/clickhouse/tables/test/replicas/replica2", recursive=True) + assert zk.exists("/clickhouse/tables/test/replicas/replica2") is None + node_1.query_and_get_error("SYSTEM RESTORE REPLICA test") + + print("Deleting replica1 path, trying to restore replica1") + zk.delete("/clickhouse/tables/test/replicas/replica1", recursive=True) + assert zk.exists("/clickhouse/tables/test/replicas/replica1") is None + + node_1.query("SYSTEM RESTART REPLICA test") + node_1.query("SYSTEM RESTORE REPLICA test") + + node_2.query("SYSTEM RESTART REPLICA test") + node_2.query("SYSTEM RESTORE REPLICA test") + + check_data(499500, 1000) + + node_1.query("INSERT INTO test SELECT number + 1000 FROM numbers(1000)") + + node_2.query("SYSTEM SYNC REPLICA test") + node_3.query("SYSTEM SYNC REPLICA test") + + check_after_restoration() diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 0ab0d57ebcf..343d8ceeca3 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -103,6 +103,7 @@ SYSTEM REPLICATION QUEUES ['SYSTEM STOP REPLICATION QUEUES','SYSTEM START REPLIC SYSTEM DROP REPLICA ['DROP REPLICA'] TABLE SYSTEM SYSTEM SYNC REPLICA ['SYNC REPLICA'] TABLE SYSTEM SYSTEM RESTART REPLICA ['RESTART REPLICA'] TABLE SYSTEM +SYSTEM RESTORE REPLICA ['RESTORE REPLICA'] TABLE SYSTEM SYSTEM FLUSH DISTRIBUTED ['FLUSH DISTRIBUTED'] TABLE SYSTEM FLUSH SYSTEM FLUSH LOGS ['FLUSH LOGS'] GLOBAL SYSTEM FLUSH SYSTEM FLUSH [] \N SYSTEM From 4dcbf3a5c5dfe1ccb2f47c025b8cb23ae71f1dd2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 20 Jun 2021 10:00:15 +0300 Subject: [PATCH 34/43] Replace __pthread_get_minstack() with a const --- .../glibc-compatibility/glibc-compatibility.c | 33 ++----------------- src/Common/ThreadStatus.cpp | 27 ++++++++------- 2 files changed, 19 insertions(+), 41 deletions(-) diff --git a/base/glibc-compatibility/glibc-compatibility.c b/base/glibc-compatibility/glibc-compatibility.c index 1ce2659a80d..e3f62b7948a 100644 --- a/base/glibc-compatibility/glibc-compatibility.c +++ b/base/glibc-compatibility/glibc-compatibility.c @@ -8,37 +8,8 @@ extern "C" { #endif -#include -#include -#include -#include - -/// glibc __pthread_get_minstack() will take TLS into account for the minimal -/// stack size (since you cannot use stack less then TLS block size, otherwise -/// some variables may be overwritten) -/// -/// So glibc implementation is: -/// -/// sysconf(_SC_PAGESIZE) + __static_tls_size + PTHREAD_STACK_MIN; -/// -/// But this helper cannot do this since: -/// - __pthread_get_minstack() is hidden in libc (note that rust tried to do -/// this but revert it to retry loop, for compatibility, while we cannot -/// use retry loop since this function is used for sigaltstack()) -/// - __static_tls_size is not exported in glibc -/// - it is not used anywhere except for clickhouse itself (for sigaltstack(), -/// to handle SIGSEGV only) and using PTHREAD_STACK_MIN (16k) is enough right -/// now. -/// -/// Also we cannot use getStackSize() (pthread_attr_getstack()) since it will -/// return 8MB, and this is too huge for signal stack. -size_t __pthread_get_minstack(const pthread_attr_t * attr) -{ - _Static_assert(PTHREAD_STACK_MIN == 16<<10, "Too small return value of __pthread_get_minstack()"); - return PTHREAD_STACK_MIN; -} - #include +#include #include #include @@ -163,6 +134,8 @@ int __open_2(const char *path, int oflag) } +#include + /// No-ops. int pthread_setname_np(pthread_t thread, const char *name) { return 0; } int pthread_getname_np(pthread_t thread, char *name, size_t len) { name[0] = '\0'; return 0; }; diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 3bacc849eb9..18f24553d5e 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -8,13 +8,7 @@ #include #include -#include - - -/// NOTE: clickhouse provide override for this function that will return -/// PTHREAD_STACK_MIN=16K (for compatibility reasons), -/// but this should be enough for signal stack. -extern "C" size_t __pthread_get_minstack(const pthread_attr_t * attr); +#include namespace DB @@ -34,22 +28,33 @@ thread_local ThreadStatus * main_thread = nullptr; namespace { +/// Alternative stack for signal handling. +/// +/// This stack should not be located in the TLS (thread local storage), since: +/// - TLS locates data on the per-thread stack +/// - And in case of stack in the signal handler will grow too much, +/// it will start overwriting TLS storage +/// (and note, that it is not too small, due to StackTrace obtaining) +/// - Plus there is no way to determine TLS block size, yes there is +/// __pthread_get_minstack() in glibc, but it is private and hence not portable. +/// +/// Also we should not use getStackSize() (pthread_attr_getstack()) since it +/// will return 8MB, and this is too huge for signal stack. struct ThreadStack { ThreadStack() - : size(__pthread_get_minstack(nullptr)) - , data(aligned_alloc(4096, size)) + : data(aligned_alloc(4096, size)) {} ~ThreadStack() { free(data); } - size_t getSize() const { return size; } + static size_t getSize() { return size; } void* getData() const { return data; } private: - size_t size; + static constexpr size_t size = 16 << 10; void *data; }; From 8a56419a9a7d828a866fb15824b9681e6199c95f Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Sun, 20 Jun 2021 10:06:12 -0400 Subject: [PATCH 35/43] Fixing alter modify tests to explicitly set the default value to '0' so that we don't have empty strings that are not guaranteed to convert to 0. --- tests/testflows/rbac/tests/privileges/alter/alter_column.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_column.py b/tests/testflows/rbac/tests/privileges/alter/alter_column.py index 91b3ab1629a..4e8bfb0b53d 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_column.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_column.py @@ -202,7 +202,7 @@ def check_modify_column_when_privilege_is_granted(table, user, node, column=None column = 'modify' with Given(f"I add the column {column}"): - node.query(f"ALTER TABLE {table} ADD COLUMN {column} String") + node.query(f"ALTER TABLE {table} ADD COLUMN {column} String DEFAULT '0'") with When(f"I insert some data into column {column}"): node.query(f"INSERT INTO {table} ({column}) VALUES ('3.4')") From 12eadc2b5edaf9e7b575e5499bd4f0005a7cb35b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 20 Jun 2021 19:54:24 +0300 Subject: [PATCH 36/43] Add a guard page for alternative signal handling stack --- src/Common/ThreadStatus.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 18f24553d5e..76b0a6797df 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -7,6 +7,7 @@ #include #include +#include #include @@ -43,10 +44,15 @@ namespace struct ThreadStack { ThreadStack() - : data(aligned_alloc(4096, size)) - {} + : data(aligned_alloc(getPageSize(), size)) + { + /// Add a guard page + /// (and since the stack grows downward, we need to protect the first page). + mprotect(data, getPageSize(), PROT_NONE); + } ~ThreadStack() { + mprotect(data, getPageSize(), PROT_WRITE|PROT_READ); free(data); } From fa29fe6ba919773a0c0daca027e5d59155963255 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 21 Jun 2021 00:39:24 +0300 Subject: [PATCH 37/43] Catch TOO_DEEP_RECURSION for formatted query too --- programs/client/Client.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 86c88b85bae..8f0f56c0b2a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1451,10 +1451,9 @@ private: } catch (Exception & e) { - if (e.code() != ErrorCodes::SYNTAX_ERROR) - { + if (e.code() != ErrorCodes::SYNTAX_ERROR && + e.code() != ErrorCodes::TOO_DEEP_RECURSION) throw; - } } if (ast_2) From feecf398fb9a6a883344a01c95fbafa76088170e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Jun 2021 11:10:44 +0300 Subject: [PATCH 38/43] Revert "Add run-id option to integration tests" --- tests/integration/ci-runner.py | 4 ++-- tests/integration/conftest.py | 8 +------- tests/integration/helpers/cluster.py | 19 ++----------------- 3 files changed, 5 insertions(+), 26 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 0af76fe2648..8b8462a7125 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -377,8 +377,8 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-ss -rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( - repo_path, image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), output_path) + cmd = "cd {}/tests/integration && ./runner --tmpfs {} -t {} {} '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( + repo_path, image_cmd, test_cmd, parallel_cmd, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: logging.info("Executing cmd: %s", cmd) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 993e7a6e973..fa14e2b06d6 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -28,10 +28,4 @@ def cleanup_environment(): logging.exception(f"cleanup_environment:{str(e)}") pass - yield - -def pytest_addoption(parser): - parser.addoption("--run-id", default="", help="run-id is used as postfix in _instances_{} directory") - -def pytest_configure(config): - os.environ['INTEGRATION_TESTS_RUN_ID'] = config.option.run_id + yield \ No newline at end of file diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7888e07587c..70a52e42048 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -203,14 +203,7 @@ class ClickHouseCluster: project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name # docker-compose removes everything non-alphanumeric from project names so we do it too. self.project_name = re.sub(r'[^a-z0-9]', '', project_name.lower()) - instances_dir_name = '_instances' - if self.name: - instances_dir_name += '_' + self.name - - if 'INTEGRATION_TESTS_RUN_ID' in os.environ: - instances_dir_name += '_' + shlex.quote(os.environ['INTEGRATION_TESTS_RUN_ID']) - - self.instances_dir = p.join(self.base_dir, instances_dir_name) + self.instances_dir = p.join(self.base_dir, '_instances' + ('' if not self.name else '_' + self.name)) self.docker_logs_path = p.join(self.instances_dir, 'docker.log') self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_variables = {} @@ -428,15 +421,7 @@ class ClickHouseCluster: pass def get_docker_handle(self, docker_id): - exception = None - for i in range(5): - try: - return self.docker_client.containers.get(docker_id) - except Exception as ex: - print("Got exception getting docker handle", str(ex)) - time.sleep(i * 2) - exception = ex - raise exception + return self.docker_client.containers.get(docker_id) def get_client_cmd(self): cmd = self.client_bin_path From 9a4125dd7cc54b97c137ea737be198e92283fe6d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 21 Jun 2021 12:06:41 +0300 Subject: [PATCH 39/43] Update ThreadStatus.cpp --- src/Common/ThreadStatus.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 76b0a6797df..1622e12712e 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -57,11 +57,11 @@ struct ThreadStack } static size_t getSize() { return size; } - void* getData() const { return data; } + void * getData() const { return data; } private: - static constexpr size_t size = 16 << 10; - void *data; + static constexpr size_t size = 16 << 10; /// 16 KiB - not too big but enough to handle error. + void * data; }; } From f37faf02fa90ef41570a71e66b99184985dee5e4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 21 Jun 2021 12:08:30 +0300 Subject: [PATCH 40/43] Update checkStackSize.cpp --- src/Common/checkStackSize.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index 9f00bbed302..b445261b304 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -26,12 +26,12 @@ static thread_local size_t max_stack_size = 0; * @param address_ - stack address * @return stack size */ -size_t getStackSize(void **address_) +size_t getStackSize(void ** out_address) { using namespace DB; size_t size; - void *address; + void * address; #if defined(OS_DARWIN) // pthread_get_stacksize_np() returns a value too low for the main thread on @@ -62,8 +62,8 @@ size_t getStackSize(void **address_) throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); #endif // OS_DARWIN - if (address_) - *address_ = address; + if (out_address) + *out_address = address; return size; } From d2c86e52e6906e88321c8accc2f2e6b5db8d595f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 21 Jun 2021 12:09:14 +0300 Subject: [PATCH 41/43] Update checkStackSize.cpp --- src/Common/checkStackSize.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index b445261b304..dfb1910a8eb 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -23,7 +23,7 @@ static thread_local void * stack_address = nullptr; static thread_local size_t max_stack_size = 0; /** - * @param address_ - stack address + * @param out_address - if not nullptr, here the address of the stack will be written. * @return stack size */ size_t getStackSize(void ** out_address) From 7d3dee172c747447d11272ad522a038212892c04 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 21 Jun 2021 12:19:25 +0000 Subject: [PATCH 42/43] FIx progress --- src/Common/ProgressIndication.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 56362015245..044d8cb1a89 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -39,7 +39,7 @@ public: void setFileProgressCallback(ContextMutablePtr context, bool write_progress_on_update = false); /// How much seconds passed since query execution start. - UInt64 elapsedSeconds() const { return watch.elapsedSeconds(); } + double elapsedSeconds() const { return watch.elapsedSeconds(); } private: /// This flag controls whether to show the progress bar. We start showing it after From eaf3f9c6e6182a42781324ac033f12d84d344f5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Jun 2021 16:59:19 +0300 Subject: [PATCH 43/43] Remove Testflows, because they are unstable --- tests/testflows/regression.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index c2e143a4b1c..661bc03509b 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -22,15 +22,15 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): tasks = [] with Pool(8) as pool: try: - run_scenario(pool, tasks, Feature(test=load("example.regression", "regression")), args) - run_scenario(pool, tasks, Feature(test=load("ldap.regression", "regression")), args) - run_scenario(pool, tasks, Feature(test=load("rbac.regression", "regression")), args) - run_scenario(pool, tasks, Feature(test=load("aes_encryption.regression", "regression")), args) - run_scenario(pool, tasks, Feature(test=load("map_type.regression", "regression")), args) - run_scenario(pool, tasks, Feature(test=load("window_functions.regression", "regression")), args) - run_scenario(pool, tasks, Feature(test=load("datetime64_extended_range.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("example.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("ldap.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("rbac.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("aes_encryption.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("map_type.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("window_functions.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("datetime64_extended_range.regression", "regression")), args) #run_scenario(pool, tasks, Feature(test=load("kerberos.regression", "regression")), args) - run_scenario(pool, tasks, Feature(test=load("extended_precision_data_types.regression", "regression")), args) + #run_scenario(pool, tasks, Feature(test=load("extended_precision_data_types.regression", "regression")), args) finally: join(tasks)