Merge branch 'master' into support-distinct-on

Signed-off-by: Zijie Lu <wslzj40@gmail.com>
This commit is contained in:
Zijie Lu 2021-06-22 19:47:17 +08:00
commit b408c7d7e6
171 changed files with 2920 additions and 975 deletions

2
.gitmodules vendored
View File

@ -103,7 +103,7 @@
url = https://github.com/ClickHouse-Extras/fastops url = https://github.com/ClickHouse-Extras/fastops
[submodule "contrib/orc"] [submodule "contrib/orc"]
path = contrib/orc path = contrib/orc
url = https://github.com/apache/orc url = https://github.com/ClickHouse-Extras/orc
[submodule "contrib/sparsehash-c11"] [submodule "contrib/sparsehash-c11"]
path = contrib/sparsehash-c11 path = contrib/sparsehash-c11
url = https://github.com/sparsehash/sparsehash-c11.git url = https://github.com/sparsehash/sparsehash-c11.git

View File

@ -2,8 +2,6 @@
#### Upgrade Notes #### Upgrade Notes
* One bug has been found after release: [#25187](https://github.com/ClickHouse/ClickHouse/issues/25187).
* Do not upgrade if you have partition key with `UUID`.
* `zstd` compression library is updated to v1.5.0. You may get messages about "checksum does not match" in replication. These messages are expected due to update of compression algorithm and you can ignore them. These messages are informational and do not indicate any kinds of undesired behaviour. * `zstd` compression library is updated to v1.5.0. You may get messages about "checksum does not match" in replication. These messages are expected due to update of compression algorithm and you can ignore them. These messages are informational and do not indicate any kinds of undesired behaviour.
* The setting `compile_expressions` is enabled by default. Although it has been heavily tested on variety of scenarios, if you find some undesired behaviour on your servers, you can try turning this setting off. * The setting `compile_expressions` is enabled by default. Although it has been heavily tested on variety of scenarios, if you find some undesired behaviour on your servers, you can try turning this setting off.
* Values of `UUID` type cannot be compared with integer. For example, instead of writing `uuid != 0` type `uuid != '00000000-0000-0000-0000-000000000000'`. * Values of `UUID` type cannot be compared with integer. For example, instead of writing `uuid != 0` type `uuid != '00000000-0000-0000-0000-000000000000'`.

View File

@ -15,4 +15,4 @@ ClickHouse® is an open-source column-oriented database management system that a
* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person. * You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person.
## Upcoming Events ## Upcoming Events
* [SF Bay Area ClickHouse Community Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/278144089/) on 16 June 2021. * [China ClickHouse Community Meetup (online)](http://hdxu.cn/rhbfZ) on 26 June 2021.

View File

@ -8,13 +8,6 @@
extern "C" { extern "C" {
#endif #endif
#include <pthread.h>
size_t __pthread_get_minstack(const pthread_attr_t * attr)
{
return 1048576; /// This is a guess. Don't sure it is correct.
}
#include <signal.h> #include <signal.h>
#include <unistd.h> #include <unistd.h>
#include <string.h> #include <string.h>
@ -141,6 +134,8 @@ int __open_2(const char *path, int oflag)
} }
#include <pthread.h>
/// No-ops. /// No-ops.
int pthread_setname_np(pthread_t thread, const char *name) { return 0; } 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; }; int pthread_getname_np(pthread_t thread, char *name, size_t len) { name[0] = '\0'; return 0; };

View File

@ -2,7 +2,7 @@
#include <errmsg.h> #include <errmsg.h>
#include <mysql.h> #include <mysql.h>
#else #else
#include <mysql/errmsg.h> #include <mysql/errmsg.h> //Y_IGNORE
#include <mysql/mysql.h> #include <mysql/mysql.h>
#endif #endif

39
base/mysqlxx/ya.make Normal file
View File

@ -0,0 +1,39 @@
# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it.
LIBRARY()
OWNER(g:clickhouse)
CFLAGS(-g0)
PEERDIR(
contrib/restricted/boost/libs
contrib/libs/libmysql_r
contrib/libs/poco/Foundation
contrib/libs/poco/Util
)
ADDINCL(
GLOBAL clickhouse/base
clickhouse/base
contrib/libs/libmysql_r
)
NO_COMPILER_WARNINGS()
NO_UTIL()
SRCS(
Connection.cpp
Exception.cpp
Pool.cpp
PoolFactory.cpp
PoolWithFailover.cpp
Query.cpp
ResultBase.cpp
Row.cpp
UseQueryResult.cpp
Value.cpp
)
END()

28
base/mysqlxx/ya.make.in Normal file
View File

@ -0,0 +1,28 @@
LIBRARY()
OWNER(g:clickhouse)
CFLAGS(-g0)
PEERDIR(
contrib/restricted/boost/libs
contrib/libs/libmysql_r
contrib/libs/poco/Foundation
contrib/libs/poco/Util
)
ADDINCL(
GLOBAL clickhouse/base
clickhouse/base
contrib/libs/libmysql_r
)
NO_COMPILER_WARNINGS()
NO_UTIL()
SRCS(
<? find . -name '*.cpp' | grep -v -F tests/ | grep -v -F examples | sed 's/^\.\// /' | sort ?>
)
END()

View File

@ -4,6 +4,7 @@ RECURSE(
common common
daemon daemon
loggers loggers
mysqlxx
pcg-random pcg-random
widechar_width widechar_width
readpassphrase readpassphrase

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit 2a1bf7d87b4a03561fc66fbb49cee8a288983c5d Subproject commit 976874b7aa7f422bf4ea595bb7d1166c617b1c26

2
contrib/arrow vendored

@ -1 +1 @@
Subproject commit 616b3dc76a0c8450b4027ded8a78e9619d7c845f Subproject commit debf751a129bdda9ff4d1e895e08957ff77000a1

View File

@ -188,6 +188,7 @@ set(ARROW_SRCS
"${LIBRARY_DIR}/array/util.cc" "${LIBRARY_DIR}/array/util.cc"
"${LIBRARY_DIR}/array/validate.cc" "${LIBRARY_DIR}/array/validate.cc"
"${LIBRARY_DIR}/compute/api_aggregate.cc"
"${LIBRARY_DIR}/compute/api_scalar.cc" "${LIBRARY_DIR}/compute/api_scalar.cc"
"${LIBRARY_DIR}/compute/api_vector.cc" "${LIBRARY_DIR}/compute/api_vector.cc"
"${LIBRARY_DIR}/compute/cast.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_basic.cc"
"${LIBRARY_DIR}/compute/kernels/aggregate_mode.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/aggregate_var_std.cc"
"${LIBRARY_DIR}/compute/kernels/codegen_internal.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_arithmetic.cc"
"${LIBRARY_DIR}/compute/kernels/scalar_boolean.cc" "${LIBRARY_DIR}/compute/kernels/scalar_boolean.cc"
"${LIBRARY_DIR}/compute/kernels/scalar_cast_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/interfaces.cc"
"${LIBRARY_DIR}/io/memory.cc" "${LIBRARY_DIR}/io/memory.cc"
"${LIBRARY_DIR}/io/slow.cc" "${LIBRARY_DIR}/io/slow.cc"
"${LIBRARY_DIR}/io/transform.cc"
"${LIBRARY_DIR}/tensor/coo_converter.cc" "${LIBRARY_DIR}/tensor/coo_converter.cc"
"${LIBRARY_DIR}/tensor/csf_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_builders.cc"
"${LIBRARY_DIR}/util/bitmap_ops.cc" "${LIBRARY_DIR}/util/bitmap_ops.cc"
"${LIBRARY_DIR}/util/bpacking.cc" "${LIBRARY_DIR}/util/bpacking.cc"
"${LIBRARY_DIR}/util/cancel.cc"
"${LIBRARY_DIR}/util/compression.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/cpu_info.cc"
"${LIBRARY_DIR}/util/decimal.cc" "${LIBRARY_DIR}/util/decimal.cc"
"${LIBRARY_DIR}/util/delimiting.cc" "${LIBRARY_DIR}/util/delimiting.cc"
@ -268,13 +270,14 @@ set(ARROW_SRCS
"${LIBRARY_DIR}/util/future.cc" "${LIBRARY_DIR}/util/future.cc"
"${LIBRARY_DIR}/util/int_util.cc" "${LIBRARY_DIR}/util/int_util.cc"
"${LIBRARY_DIR}/util/io_util.cc" "${LIBRARY_DIR}/util/io_util.cc"
"${LIBRARY_DIR}/util/iterator.cc"
"${LIBRARY_DIR}/util/key_value_metadata.cc" "${LIBRARY_DIR}/util/key_value_metadata.cc"
"${LIBRARY_DIR}/util/logging.cc" "${LIBRARY_DIR}/util/logging.cc"
"${LIBRARY_DIR}/util/memory.cc" "${LIBRARY_DIR}/util/memory.cc"
"${LIBRARY_DIR}/util/mutex.cc"
"${LIBRARY_DIR}/util/string_builder.cc" "${LIBRARY_DIR}/util/string_builder.cc"
"${LIBRARY_DIR}/util/string.cc" "${LIBRARY_DIR}/util/string.cc"
"${LIBRARY_DIR}/util/task_group.cc" "${LIBRARY_DIR}/util/task_group.cc"
"${LIBRARY_DIR}/util/tdigest.cc"
"${LIBRARY_DIR}/util/thread_pool.cc" "${LIBRARY_DIR}/util/thread_pool.cc"
"${LIBRARY_DIR}/util/time.cc" "${LIBRARY_DIR}/util/time.cc"
"${LIBRARY_DIR}/util/trie.cc" "${LIBRARY_DIR}/util/trie.cc"
@ -368,14 +371,14 @@ set(PARQUET_SRCS
"${LIBRARY_DIR}/column_reader.cc" "${LIBRARY_DIR}/column_reader.cc"
"${LIBRARY_DIR}/column_scanner.cc" "${LIBRARY_DIR}/column_scanner.cc"
"${LIBRARY_DIR}/column_writer.cc" "${LIBRARY_DIR}/column_writer.cc"
"${LIBRARY_DIR}/deprecated_io.cc"
"${LIBRARY_DIR}/encoding.cc" "${LIBRARY_DIR}/encoding.cc"
"${LIBRARY_DIR}/encryption.cc" "${LIBRARY_DIR}/encryption/encryption.cc"
"${LIBRARY_DIR}/encryption_internal.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_reader.cc"
"${LIBRARY_DIR}/file_writer.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_conversion.cc"
"${LIBRARY_DIR}/level_comparison.cc" "${LIBRARY_DIR}/level_comparison.cc"
"${LIBRARY_DIR}/metadata.cc" "${LIBRARY_DIR}/metadata.cc"
@ -385,6 +388,8 @@ set(PARQUET_SRCS
"${LIBRARY_DIR}/properties.cc" "${LIBRARY_DIR}/properties.cc"
"${LIBRARY_DIR}/schema.cc" "${LIBRARY_DIR}/schema.cc"
"${LIBRARY_DIR}/statistics.cc" "${LIBRARY_DIR}/statistics.cc"
"${LIBRARY_DIR}/stream_reader.cc"
"${LIBRARY_DIR}/stream_writer.cc"
"${LIBRARY_DIR}/types.cc" "${LIBRARY_DIR}/types.cc"
"${GEN_LIBRARY_DIR}/parquet_constants.cpp" "${GEN_LIBRARY_DIR}/parquet_constants.cpp"

2
contrib/flatbuffers vendored

@ -1 +1 @@
Subproject commit 22e3ffc66d2d7d72d1414390aa0f04ffd114a5a1 Subproject commit eb3f827948241ce0e701516f16cd67324802bce9

2
contrib/orc vendored

@ -1 +1 @@
Subproject commit 5981208e39447df84827f6a961d1da76bacb6078 Subproject commit 0a936f6bbdb9303308973073f8623b5a8d82eae1

2
contrib/replxx vendored

@ -1 +1 @@
Subproject commit 2b24f14594d7606792b92544bb112a6322ba34d7 Subproject commit c81be6c68b146f15f2096b7ef80e3f21fe27004c

View File

@ -1591,6 +1591,18 @@ FORMAT PrettyCompactMonoBlock
Default value: 0 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} ## 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. Limit for number of sharding key values, turns off `optimize_skip_unused_shards` if the limit is reached.

View File

@ -38,6 +38,7 @@ The list of available `SYSTEM` statements:
- [START REPLICATION QUEUES](#query_language-system-start-replication-queues) - [START REPLICATION QUEUES](#query_language-system-start-replication-queues)
- [SYNC REPLICA](#query_language-system-sync-replica) - [SYNC REPLICA](#query_language-system-sync-replica)
- [RESTART REPLICA](#query_language-system-restart-replica) - [RESTART REPLICA](#query_language-system-restart-replica)
- [RESTORE REPLICA](#query_language-system-restore-replica)
- [RESTART REPLICAS](#query_language-system-restart-replicas) - [RESTART REPLICAS](#query_language-system-restart-replicas)
## RELOAD EMBEDDED DICTIONARIES {#query_language-system-reload-emdedded-dictionaries} ## 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} ### 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 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. 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 ``` sql
SYSTEM RESTART REPLICA [db.]replicated_merge_tree_family_table_name 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} ### 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 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

File diff suppressed because it is too large Load Diff

View File

@ -33,6 +33,7 @@ option (ENABLE_CLICKHOUSE_OBFUSCATOR "Table data obfuscator (convert real data t
${ENABLE_CLICKHOUSE_ALL}) ${ENABLE_CLICKHOUSE_ALL})
# https://clickhouse.tech/docs/en/operations/utilities/odbc-bridge/ # https://clickhouse.tech/docs/en/operations/utilities/odbc-bridge/
# TODO Also needs NANODBC.
if (ENABLE_ODBC) if (ENABLE_ODBC)
option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "HTTP-server working like a proxy to ODBC driver" option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "HTTP-server working like a proxy to ODBC driver"
${ENABLE_CLICKHOUSE_ALL}) ${ENABLE_CLICKHOUSE_ALL})

View File

@ -20,6 +20,7 @@ CLICKHOUSE_QueryProcessingStage=(
fetch_columns fetch_columns
with_mergeable_state with_mergeable_state
with_mergeable_state_after_aggregation with_mergeable_state_after_aggregation
with_mergeable_state_after_aggregation_and_limit
) )
CLICKHOUSE_Format=( CLICKHOUSE_Format=(

View File

@ -580,7 +580,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
("query", value<std::string>()->default_value(""), "query to execute") ("query", value<std::string>()->default_value(""), "query to execute")
("concurrency,c", value<unsigned>()->default_value(1), "number of parallel queries") ("concurrency,c", value<unsigned>()->default_value(1), "number of parallel queries")
("delay,d", value<double>()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)") ("delay,d", value<double>()->default_value(1), "delay between intermediate reports in seconds (set 0 to disable reports)")
("stage", value<std::string>()->default_value("complete"), "request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation") ("stage", value<std::string>()->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<size_t>()->default_value(0), "amount of queries to be executed") ("iterations,i", value<size_t>()->default_value(0), "amount of queries to be executed")
("timelimit,t", value<double>()->default_value(0.), "stop launch of queries after specified time limit") ("timelimit,t", value<double>()->default_value(0.), "stop launch of queries after specified time limit")
("randomize,r", value<bool>()->default_value(false), "randomize order of execution") ("randomize,r", value<bool>()->default_value(false), "randomize order of execution")

View File

@ -29,7 +29,6 @@
#include <common/find_symbols.h> #include <common/find_symbols.h>
#include <common/LineReader.h> #include <common/LineReader.h>
#include <Common/ClickHouseRevision.h> #include <Common/ClickHouseRevision.h>
#include <Common/Stopwatch.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ShellCommand.h> #include <Common/ShellCommand.h>
#include <Common/UnicodeBar.h> #include <Common/UnicodeBar.h>
@ -85,7 +84,7 @@
#include <common/argsToConfig.h> #include <common/argsToConfig.h>
#include <Common/TerminalSize.h> #include <Common/TerminalSize.h>
#include <Common/UTF8Helpers.h> #include <Common/UTF8Helpers.h>
#include <Common/ProgressBar.h> #include <Common/ProgressIndication.h>
#include <filesystem> #include <filesystem>
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
@ -113,6 +112,7 @@ namespace ErrorCodes
extern const int DEADLOCK_AVOIDED; extern const int DEADLOCK_AVOIDED;
extern const int UNRECOGNIZED_ARGUMENTS; extern const int UNRECOGNIZED_ARGUMENTS;
extern const int SYNTAX_ERROR; extern const int SYNTAX_ERROR;
extern const int TOO_DEEP_RECURSION;
} }
@ -230,13 +230,13 @@ private:
String server_version; String server_version;
String server_display_name; String server_display_name;
Stopwatch watch; /// true by default - for interactive mode, might be changed when --progress option is checked for
/// non-interactive mode.
bool need_render_progress = true;
/// The server periodically sends information about how much data was read since last time. bool written_first_block = false;
Progress progress;
/// Progress bar ProgressIndication progress_indication;
ProgressBar progress_bar;
/// External tables info. /// External tables info.
std::list<ExternalTable> external_tables; std::list<ExternalTable> external_tables;
@ -536,7 +536,7 @@ private:
if (!is_interactive) if (!is_interactive)
{ {
progress_bar.need_render_progress = config().getBool("progress", false); need_render_progress = config().getBool("progress", false);
echo_queries = config().getBool("echo", false); echo_queries = config().getBool("echo", false);
ignore_error = config().getBool("ignore-error", false); ignore_error = config().getBool("ignore-error", false);
} }
@ -1268,7 +1268,8 @@ private:
} }
catch (const Exception & e) catch (const Exception & e)
{ {
if (e.code() != ErrorCodes::SYNTAX_ERROR) if (e.code() != ErrorCodes::SYNTAX_ERROR &&
e.code() != ErrorCodes::TOO_DEEP_RECURSION)
throw; throw;
} }
@ -1450,11 +1451,10 @@ private:
} }
catch (Exception & e) catch (Exception & e)
{ {
if (e.code() != ErrorCodes::SYNTAX_ERROR) if (e.code() != ErrorCodes::SYNTAX_ERROR &&
{ e.code() != ErrorCodes::TOO_DEEP_RECURSION)
throw; throw;
} }
}
if (ast_2) if (ast_2)
{ {
@ -1578,12 +1578,9 @@ private:
} }
} }
watch.restart();
processed_rows = 0; processed_rows = 0;
progress.reset(); written_first_block = false;
progress_bar.show_progress_bar = false; progress_indication.resetProgress();
progress_bar.written_progress_chars = 0;
progress_bar.written_first_block = false;
{ {
/// Temporarily apply query settings to context. /// Temporarily apply query settings to context.
@ -1651,16 +1648,15 @@ private:
if (is_interactive) if (is_interactive)
{ {
std::cout << std::endl << processed_rows << " rows in set. Elapsed: " << watch.elapsedSeconds() << " sec. "; std::cout << std::endl << processed_rows << " rows in set. Elapsed: " << progress_indication.elapsedSeconds() << " sec. ";
/// Write final progress if it makes sense to do so.
if (progress.read_rows >= 1000)
writeFinalProgress(); writeFinalProgress();
std::cout << std::endl << std::endl; std::cout << std::endl << std::endl;
} }
else if (print_time_to_stderr) else if (print_time_to_stderr)
{ {
std::cerr << watch.elapsedSeconds() << "\n"; std::cerr << progress_indication.elapsedSeconds() << "\n";
} }
} }
@ -1835,6 +1831,19 @@ private:
/// Send data read from stdin. /// Send data read from stdin.
try try
{ {
if (need_render_progress)
{
/// Set total_bytes_to_read for current fd.
FileProgress file_progress(0, std_in.size());
progress_indication.updateProgress(Progress(file_progress));
/// Set callback to be called on file progress.
progress_indication.setFileProgressCallback(context, true);
/// Add callback to track reading from fd.
std_in.setProgressCallback(context);
}
sendDataFrom(std_in, sample, columns_description); sendDataFrom(std_in, sample, columns_description);
} }
catch (Exception & e) catch (Exception & e)
@ -1957,7 +1966,7 @@ private:
cancelled = true; cancelled = true;
if (is_interactive) if (is_interactive)
{ {
progress_bar.clearProgress(); progress_indication.clearProgressOutput();
std::cout << "Cancelling query." << std::endl; std::cout << "Cancelling query." << std::endl;
} }
@ -2184,7 +2193,7 @@ private:
current_format = "Vertical"; current_format = "Vertical";
/// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly. /// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly.
if (!progress_bar.need_render_progress) if (!need_render_progress)
block_out_stream = context->getOutputStreamParallelIfPossible(current_format, *out_buf, block); block_out_stream = context->getOutputStreamParallelIfPossible(current_format, *out_buf, block);
else else
block_out_stream = context->getOutputStream(current_format, *out_buf, block); block_out_stream = context->getOutputStream(current_format, *out_buf, block);
@ -2243,25 +2252,25 @@ private:
if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100))
return; return;
if (progress_bar.need_render_progress) if (need_render_progress)
progress_bar.clearProgress(); progress_indication.clearProgressOutput();
block_out_stream->write(block); block_out_stream->write(block);
progress_bar.written_first_block = true; written_first_block = true;
/// Received data block is immediately displayed to the user. /// Received data block is immediately displayed to the user.
block_out_stream->flush(); block_out_stream->flush();
/// Restore progress bar after data block. /// Restore progress bar after data block.
if (progress_bar.need_render_progress) if (need_render_progress)
progress_bar.writeProgress(progress, watch.elapsed()); progress_indication.writeProgress();
} }
void onLogData(Block & block) void onLogData(Block & block)
{ {
initLogsOutputStream(); initLogsOutputStream();
progress_bar.clearProgress(); progress_indication.clearProgressOutput();
logs_out_stream->write(block); logs_out_stream->write(block);
logs_out_stream->flush(); logs_out_stream->flush();
} }
@ -2282,28 +2291,23 @@ private:
void onProgress(const Progress & value) void onProgress(const Progress & value)
{ {
if (!progress_bar.updateProgress(progress, value)) if (!progress_indication.updateProgress(value))
{ {
// Just a keep-alive update. // Just a keep-alive update.
return; return;
} }
if (block_out_stream) if (block_out_stream)
block_out_stream->onProgress(value); block_out_stream->onProgress(value);
progress_bar.writeProgress(progress, watch.elapsed());
if (need_render_progress)
progress_indication.writeProgress();
} }
void writeFinalProgress() void writeFinalProgress()
{ {
std::cout << "Processed " << formatReadableQuantity(progress.read_rows) << " rows, " progress_indication.writeFinalProgress();
<< formatReadableSizeWithDecimalSuffix(progress.read_bytes);
size_t elapsed_ns = watch.elapsed();
if (elapsed_ns)
std::cout << " (" << formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., "
<< formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)";
else
std::cout << ". ";
} }
@ -2324,7 +2328,7 @@ private:
void onEndOfStream() void onEndOfStream()
{ {
progress_bar.clearProgress(); progress_indication.clearProgressOutput();
if (block_out_stream) if (block_out_stream)
block_out_stream->writeSuffix(); block_out_stream->writeSuffix();
@ -2334,9 +2338,9 @@ private:
resetOutput(); resetOutput();
if (is_interactive && !progress_bar.written_first_block) if (is_interactive && !written_first_block)
{ {
progress_bar.clearProgress(); progress_indication.clearProgressOutput();
std::cout << "Ok." << std::endl; std::cout << "Ok." << std::endl;
} }
} }
@ -2468,7 +2472,7 @@ public:
("password", po::value<std::string>()->implicit_value("\n", ""), "password") ("password", po::value<std::string>()->implicit_value("\n", ""), "password")
("ask-password", "ask-password") ("ask-password", "ask-password")
("quota_key", po::value<std::string>(), "A string to differentiate quotas when the user have keyed quotas configured on server") ("quota_key", po::value<std::string>(), "A string to differentiate quotas when the user have keyed quotas configured on server")
("stage", po::value<std::string>()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation") ("stage", po::value<std::string>()->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<std::string>(), "query_id") ("query_id", po::value<std::string>(), "query_id")
("query,q", po::value<std::string>(), "query") ("query,q", po::value<std::string>(), "query")
("database,d", po::value<std::string>(), "database") ("database,d", po::value<std::string>(), "database")

View File

@ -286,7 +286,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf
+ "." + escapeForFileName(table_push.first) + "." + escapeForFileName(table_push.first)
+ "." + escapeForFileName(table_push.second); + "." + escapeForFileName(table_push.second);
engine_push_str = config.getString(table_prefix + "engine"); engine_push_str = config.getString(table_prefix + "engine", "rand()");
{ {
ParserStorage parser_storage; ParserStorage parser_storage;

View File

@ -389,32 +389,29 @@ void LocalServer::processQueries()
CurrentThread::QueryScope query_scope_holder(context); CurrentThread::QueryScope query_scope_holder(context);
///Set progress show ///Set progress show
progress_bar.need_render_progress = config().getBool("progress", false); need_render_progress = config().getBool("progress", false);
if (progress_bar.need_render_progress) if (need_render_progress)
{ {
context->setProgressCallback([&](const Progress & value) context->setProgressCallback([&](const Progress & value)
{ {
if (!progress_bar.updateProgress(progress, value)) /// Write progress only if progress was updated
{ if (progress_indication.updateProgress(value))
// Just a keep-alive update. progress_indication.writeProgress();
return;
}
progress_bar.writeProgress(progress, watch.elapsed());
}); });
} }
bool echo_queries = config().hasOption("echo") || config().hasOption("verbose"); bool echo_queries = config().hasOption("echo") || config().hasOption("verbose");
if (need_render_progress)
progress_indication.setFileProgressCallback(context);
std::exception_ptr exception; std::exception_ptr exception;
for (const auto & query : queries) for (const auto & query : queries)
{ {
watch.restart(); written_first_block = false;
progress.reset(); progress_indication.resetProgress();
progress_bar.show_progress_bar = false;
progress_bar.written_progress_chars = 0;
progress_bar.written_first_block = false;
ReadBufferFromString read_buf(query); ReadBufferFromString read_buf(query);
WriteBufferFromFileDescriptor write_buf(STDOUT_FILENO); WriteBufferFromFileDescriptor write_buf(STDOUT_FILENO);

View File

@ -7,7 +7,7 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <loggers/Loggers.h> #include <loggers/Loggers.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
#include <Common/ProgressBar.h> #include <Common/ProgressIndication.h>
namespace DB namespace DB
{ {
@ -49,9 +49,12 @@ protected:
/// Settings specified via command line args /// Settings specified via command line args
Settings cmd_settings; Settings cmd_settings;
ProgressBar progress_bar;
Progress progress; bool need_render_progress = false;
Stopwatch watch;
bool written_first_block = false;
ProgressIndication progress_indication;
std::optional<std::filesystem::path> temporary_directory_to_delete; std::optional<std::filesystem::path> temporary_directory_to_delete;
}; };

View File

@ -132,7 +132,7 @@ void ODBCBlockInputStream::insertValue(
auto value = row.get<std::string>(idx); auto value = row.get<std::string>(idx);
ReadBufferFromString in(value); ReadBufferFromString in(value);
time_t time = 0; time_t time = 0;
readDateTimeText(time, in); readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(data_type.get())->getTimeZone());
if (time < 0) if (time < 0)
time = 0; time = 0;
assert_cast<ColumnUInt32 &>(column).insertValue(time); assert_cast<ColumnUInt32 &>(column).insertValue(time);

View File

@ -154,6 +154,7 @@ enum class AccessType
M(SYSTEM_DROP_REPLICA, "DROP REPLICA", TABLE, SYSTEM) \ M(SYSTEM_DROP_REPLICA, "DROP REPLICA", TABLE, SYSTEM) \
M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \ M(SYSTEM_SYNC_REPLICA, "SYNC REPLICA", TABLE, SYSTEM) \
M(SYSTEM_RESTART_REPLICA, "RESTART 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_DISTRIBUTED, "FLUSH DISTRIBUTED", TABLE, SYSTEM_FLUSH) \
M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \ M(SYSTEM_FLUSH_LOGS, "FLUSH LOGS", GLOBAL, SYSTEM_FLUSH) \
M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \ M(SYSTEM_FLUSH, "", GROUP, SYSTEM) \

View File

@ -191,6 +191,7 @@ public:
void nestedRemoveNullable() { dictionary.getColumnUnique().nestedRemoveNullable(); } void nestedRemoveNullable() { dictionary.getColumnUnique().nestedRemoveNullable(); }
const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); } const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); }
IColumnUnique & getDictionary() { return dictionary.getColumnUnique(); }
const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); } const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); }
/// IColumnUnique & getUnique() { return static_cast<IColumnUnique &>(*column_unique); } /// IColumnUnique & getUnique() { return static_cast<IColumnUnique &>(*column_unique); }
/// ColumnPtr getUniquePtr() const { return column_unique; } /// ColumnPtr getUniquePtr() const { return column_unique; }

View File

@ -664,8 +664,8 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
{ {
fs::path preprocessed_configs_path("preprocessed_configs/"); fs::path preprocessed_configs_path("preprocessed_configs/");
auto new_path = loaded_config.config_path; auto new_path = loaded_config.config_path;
if (new_path.substr(0, main_config_path.size()) == main_config_path) if (new_path.starts_with(main_config_path))
new_path.replace(0, main_config_path.size(), ""); new_path.erase(0, main_config_path.size());
std::replace(new_path.begin(), new_path.end(), '/', '_'); std::replace(new_path.begin(), new_path.end(), '/', '_');
if (preprocessed_dir.empty()) if (preprocessed_dir.empty())
@ -708,6 +708,8 @@ void ConfigProcessor::savePreprocessedConfig(const LoadedConfig & loaded_config,
void ConfigProcessor::setConfigPath(const std::string & config_path) void ConfigProcessor::setConfigPath(const std::string & config_path)
{ {
main_config_path = config_path; main_config_path = config_path;
if (!main_config_path.ends_with('/'))
main_config_path += '/';
} }
} }

View File

@ -554,7 +554,8 @@
M(584, PROJECTION_NOT_USED) \ M(584, PROJECTION_NOT_USED) \
M(585, CANNOT_PARSE_YAML) \ M(585, CANNOT_PARSE_YAML) \
M(586, CANNOT_CREATE_FILE) \ M(586, CANNOT_CREATE_FILE) \
M(587, DISTINCT_ON_AND_LIMIT_BY_TOGETHER) \ M(587, CONCURRENT_ACCESS_NOT_SUPPORTED) \
M(588, DISTINCT_ON_AND_LIMIT_BY_TOGETHER) \
\ \
M(998, POSTGRESQL_CONNECTION_FAILURE) \ M(998, POSTGRESQL_CONNECTION_FAILURE) \
M(999, KEEPER_EXCEPTION) \ M(999, KEEPER_EXCEPTION) \

View File

@ -1,32 +0,0 @@
#pragma once
#include <Common/Stopwatch.h>
#include <IO/Progress.h>
/// http://en.wikipedia.org/wiki/ANSI_escape_code
#define CLEAR_TO_END_OF_LINE "\033[K"
namespace DB
{
struct ProgressBar
{
public:
static bool updateProgress(Progress & progress, const Progress & value);
void writeProgress(const Progress & progress, const size_t elapsed_ns);
void clearProgress();
/// For interactive mode always show progress bar, for non-interactive mode it is accessed from config().
bool need_render_progress = true;
bool show_progress_bar = false;
size_t written_progress_chars = 0;
bool written_first_block = false;
bool clear_progress = false;
};
}

View File

@ -1,22 +1,69 @@
#include "ProgressBar.h" #include "ProgressIndication.h"
#include <IO/WriteBufferFromFileDescriptor.h> #include <IO/WriteBufferFromFileDescriptor.h>
#include <Common/TerminalSize.h> #include <Common/TerminalSize.h>
#include <Common/UnicodeBar.h> #include <Common/UnicodeBar.h>
#include <Databases/DatabaseMemory.h> #include <Databases/DatabaseMemory.h>
/// FIXME: progress bar in clickhouse-local needs to be cleared after query execution
/// - same as it is now in clickhouse-client. Also there is no writeFinalProgress call
/// in clickhouse-local.
namespace DB namespace DB
{ {
bool ProgressBar::updateProgress(Progress & progress, const Progress & value) bool ProgressIndication::updateProgress(const Progress & value)
{ {
return progress.incrementPiecewiseAtomically(value); return progress.incrementPiecewiseAtomically(value);
} }
void ProgressBar::writeProgress(const Progress & progress, const size_t elapsed_ns) void ProgressIndication::clearProgressOutput()
{ {
if (!need_render_progress) if (written_progress_chars)
{
written_progress_chars = 0;
std::cerr << "\r" CLEAR_TO_END_OF_LINE;
}
}
void ProgressIndication::resetProgress()
{
watch.restart();
progress.reset();
show_progress_bar = false;
written_progress_chars = 0;
write_progress_on_update = false;
}
void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool write_progress_on_update_)
{
write_progress_on_update = write_progress_on_update_;
context->setFileProgressCallback([&](const FileProgress & file_progress)
{
progress.incrementPiecewiseAtomically(Progress(file_progress));
if (write_progress_on_update)
writeProgress();
});
}
void ProgressIndication::writeFinalProgress()
{
if (progress.read_rows < 1000)
return; return;
std::cout << "Processed " << formatReadableQuantity(progress.read_rows) << " rows, "
<< formatReadableSizeWithDecimalSuffix(progress.read_bytes);
size_t elapsed_ns = watch.elapsed();
if (elapsed_ns)
std::cout << " (" << formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., "
<< formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)";
else
std::cout << ". ";
}
void ProgressIndication::writeProgress()
{
/// Output all progress bar commands to stderr at once to avoid flicker. /// Output all progress bar commands to stderr at once to avoid flicker.
WriteBufferFromFileDescriptor message(STDERR_FILENO, 1024); WriteBufferFromFileDescriptor message(STDERR_FILENO, 1024);
@ -45,26 +92,37 @@ void ProgressBar::writeProgress(const Progress & progress, const size_t elapsed_
message << '\r'; message << '\r';
size_t prefix_size = message.count(); size_t prefix_size = message.count();
size_t read_bytes = progress.read_raw_bytes ? progress.read_raw_bytes : progress.read_bytes;
message << indicator << " Progress: "; message << indicator << " Progress: ";
message message
<< formatReadableQuantity(progress.read_rows) << " rows, " << formatReadableQuantity(progress.read_rows) << " rows, "
<< formatReadableSizeWithDecimalSuffix(progress.read_bytes); << formatReadableSizeWithDecimalSuffix(read_bytes);
auto elapsed_ns = watch.elapsed();
if (elapsed_ns) if (elapsed_ns)
message << " (" message << " ("
<< formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., " << formatReadableQuantity(progress.read_rows * 1000000000.0 / elapsed_ns) << " rows/s., "
<< formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.) "; << formatReadableSizeWithDecimalSuffix(read_bytes * 1000000000.0 / elapsed_ns) << "/s.) ";
else else
message << ". "; message << ". ";
written_progress_chars = message.count() - prefix_size - (strlen(indicator) - 2); /// Don't count invisible output (escape sequences). written_progress_chars = message.count() - prefix_size - (strlen(indicator) - 2); /// Don't count invisible output (escape sequences).
/// If the approximate number of rows to process is known, we can display a progress bar and percentage. /// If the approximate number of rows to process is known, we can display a progress bar and percentage.
if (progress.total_rows_to_read > 0) if (progress.total_rows_to_read || progress.total_raw_bytes_to_read)
{ {
size_t total_rows_corrected = std::max(progress.read_rows, progress.total_rows_to_read); size_t current_count, max_count;
if (progress.total_rows_to_read)
{
current_count = progress.read_rows;
max_count = std::max(progress.read_rows, progress.total_rows_to_read);
}
else
{
current_count = progress.read_raw_bytes;
max_count = std::max(progress.read_raw_bytes, progress.total_raw_bytes_to_read);
}
/// To avoid flicker, display progress bar only if .5 seconds have passed since query execution start /// To avoid flicker, display progress bar only if .5 seconds have passed since query execution start
/// and the query is less than halfway done. /// and the query is less than halfway done.
@ -72,7 +130,7 @@ void ProgressBar::writeProgress(const Progress & progress, const size_t elapsed_
if (elapsed_ns > 500000000) if (elapsed_ns > 500000000)
{ {
/// Trigger to start displaying progress bar. If query is mostly done, don't display it. /// Trigger to start displaying progress bar. If query is mostly done, don't display it.
if (progress.read_rows * 2 < total_rows_corrected) if (current_count * 2 < max_count)
show_progress_bar = true; show_progress_bar = true;
if (show_progress_bar) if (show_progress_bar)
@ -81,7 +139,7 @@ void ProgressBar::writeProgress(const Progress & progress, const size_t elapsed_
if (width_of_progress_bar > 0) if (width_of_progress_bar > 0)
{ {
std::string bar std::string bar
= UnicodeBar::render(UnicodeBar::getWidth(progress.read_rows, 0, total_rows_corrected, width_of_progress_bar)); = UnicodeBar::render(UnicodeBar::getWidth(current_count, 0, max_count, width_of_progress_bar));
message << "\033[0;32m" << bar << "\033[0m"; message << "\033[0;32m" << bar << "\033[0m";
if (width_of_progress_bar > static_cast<ssize_t>(bar.size() / UNICODE_BAR_CHAR_SIZE)) if (width_of_progress_bar > static_cast<ssize_t>(bar.size() / UNICODE_BAR_CHAR_SIZE))
message << std::string(width_of_progress_bar - bar.size() / UNICODE_BAR_CHAR_SIZE, ' '); message << std::string(width_of_progress_bar - bar.size() / UNICODE_BAR_CHAR_SIZE, ' ');
@ -90,7 +148,7 @@ void ProgressBar::writeProgress(const Progress & progress, const size_t elapsed_
} }
/// Underestimate percentage a bit to avoid displaying 100%. /// Underestimate percentage a bit to avoid displaying 100%.
message << ' ' << (99 * progress.read_rows / total_rows_corrected) << '%'; message << ' ' << (99 * current_count / max_count) << '%';
} }
message << CLEAR_TO_END_OF_LINE; message << CLEAR_TO_END_OF_LINE;
@ -99,13 +157,4 @@ void ProgressBar::writeProgress(const Progress & progress, const size_t elapsed_
message.next(); message.next();
} }
void ProgressBar::clearProgress()
{
if (written_progress_chars)
{
written_progress_chars = 0;
std::cerr << "\r" CLEAR_TO_END_OF_LINE;
}
}
} }

View File

@ -0,0 +1,63 @@
#pragma once
#include <IO/Progress.h>
#include <Interpreters/Context.h>
#include <Common/Stopwatch.h>
/// http://en.wikipedia.org/wiki/ANSI_escape_code
#define CLEAR_TO_END_OF_LINE "\033[K"
namespace DB
{
class ProgressIndication
{
public:
/// Write progress to stderr.
void writeProgress();
void writeFinalProgress();
/// Clear stderr output.
void clearProgressOutput();
/// Reset progress values.
void resetProgress();
/// Update Progress object. It can be updated from:
/// 1. onProgress in clickhouse-client;
/// 2. ProgressCallback via setProgressCallback methrod in:
/// - context (used in clickhouse-local, can also be added in arbitrary place)
/// - SourceWithProgress (also in streams)
/// - readBufferFromFileDescriptor (for file processing progress)
bool updateProgress(const Progress & value);
/// In some cases there is a need to update progress value, when there is no access to progress_inidcation object.
/// In this case it is added via context.
/// `write_progress_on_update` is needed to write progress for loading files data via pipe in non-interactive mode.
void setFileProgressCallback(ContextMutablePtr context, bool write_progress_on_update = false);
/// How much seconds passed since query execution start.
double elapsedSeconds() const { return watch.elapsedSeconds(); }
private:
/// This flag controls whether to show the progress bar. We start showing it after
/// the query has been executing for 0.5 seconds, and is still less than half complete.
bool show_progress_bar = false;
/// Width of how much has been printed currently into stderr. Used to define size of progress bar and
/// to check whether progress output needs to be cleared.
size_t written_progress_chars = 0;
/// The server periodically sends information about how much data was read since last time.
/// This information is stored here.
Progress progress;
/// Track query execution time.
Stopwatch watch;
bool write_progress_on_update = false;
};
}

View File

@ -15,16 +15,19 @@ namespace DB::ErrorCodes
uint16_t getTerminalWidth() uint16_t getTerminalWidth()
{ {
struct winsize terminal_size {};
if (isatty(STDIN_FILENO)) if (isatty(STDIN_FILENO))
{ {
struct winsize terminal_size {};
if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size)) if (ioctl(STDIN_FILENO, TIOCGWINSZ, &terminal_size))
DB::throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", DB::ErrorCodes::SYSTEM_ERROR); DB::throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", DB::ErrorCodes::SYSTEM_ERROR);
return terminal_size.ws_col;
} }
return 0; else if (isatty(STDERR_FILENO))
{
if (ioctl(STDERR_FILENO, TIOCGWINSZ, &terminal_size))
DB::throwFromErrno("Cannot obtain terminal window size (ioctl TIOCGWINSZ)", DB::ErrorCodes::SYSTEM_ERROR);
}
/// Default - 0.
return terminal_size.ws_col;
} }
po::options_description createOptionsDescription(const std::string & caption, uint16_t terminal_width) po::options_description createOptionsDescription(const std::string & caption, uint16_t terminal_width)

View File

@ -7,8 +7,9 @@
#include <Poco/Logger.h> #include <Poco/Logger.h>
#include <common/getThreadId.h> #include <common/getThreadId.h>
#include <common/getPageSize.h>
#include <signal.h> #include <csignal>
namespace DB namespace DB
@ -25,8 +26,48 @@ thread_local ThreadStatus * current_thread = nullptr;
thread_local ThreadStatus * main_thread = nullptr; thread_local ThreadStatus * main_thread = nullptr;
#if !defined(SANITIZER) && !defined(ARCADIA_BUILD) #if !defined(SANITIZER) && !defined(ARCADIA_BUILD)
alignas(4096) static thread_local char alt_stack[std::max<size_t>(MINSIGSTKSZ, 4096)]; namespace
static thread_local bool has_alt_stack = false; {
/// 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()
: 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);
}
static size_t getSize() { return size; }
void * getData() const { return data; }
private:
static constexpr size_t size = 16 << 10; /// 16 KiB - not too big but enough to handle error.
void * data;
};
}
static thread_local ThreadStack alt_stack;
static thread_local bool has_alt_stack = false;
#endif #endif
@ -54,9 +95,9 @@ ThreadStatus::ThreadStatus()
/// We have to call 'sigaltstack' before first 'sigaction'. (It does not work other way, for unknown reason). /// We have to call 'sigaltstack' before first 'sigaction'. (It does not work other way, for unknown reason).
stack_t altstack_description{}; 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_flags = 0;
altstack_description.ss_size = sizeof(alt_stack); altstack_description.ss_size = alt_stack.getSize();
if (0 != sigaltstack(&altstack_description, nullptr)) if (0 != sigaltstack(&altstack_description, nullptr))
{ {

View File

@ -22,19 +22,17 @@ namespace DB
static thread_local void * stack_address = nullptr; static thread_local void * stack_address = nullptr;
static thread_local size_t max_stack_size = 0; static thread_local size_t max_stack_size = 0;
/** 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. * @param out_address - if not nullptr, here the address of the stack will be written.
* If application is using user-space lightweight threads with manually allocated stacks, * @return stack size
* current implementation is not reasonable, as it has no way to properly check the remaining
* stack size without knowing the details of how stacks are allocated.
* We mark this function as weak symbol to be able to replace it in another ClickHouse-based products.
*/ */
__attribute__((__weak__)) void checkStackSize() size_t getStackSize(void ** out_address)
{ {
using namespace DB; using namespace DB;
if (!stack_address) size_t size;
{ void * address;
#if defined(OS_DARWIN) #if defined(OS_DARWIN)
// pthread_get_stacksize_np() returns a value too low for the main thread on // 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 // OSX 10.9, http://mail.openjdk.java.net/pipermail/hotspot-dev/2013-October/011369.html
@ -43,10 +41,10 @@ __attribute__((__weak__)) void checkStackSize()
// https://developer.apple.com/library/mac/documentation/Cocoa/Conceptual/Multithreading/CreatingThreads/CreatingThreads.html // 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. // Stack size for the main thread is 8MB on OSX excluding the guard page size.
pthread_t thread = pthread_self(); pthread_t thread = pthread_self();
max_stack_size = pthread_main_np() ? (8 * 1024 * 1024) : pthread_get_stacksize_np(thread); 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 points to the start of the stack, not the end how it's returned by pthread_get_stackaddr_np
stack_address = reinterpret_cast<void*>(reinterpret_cast<uintptr_t>(pthread_get_stackaddr_np(thread)) - max_stack_size); address = reinterpret_cast<void*>(reinterpret_cast<uintptr_t>(pthread_get_stackaddr_np(thread)) - max_stack_size);
#else #else
pthread_attr_t attr; pthread_attr_t attr;
# if defined(__FreeBSD__) || defined(OS_SUNOS) # if defined(__FreeBSD__) || defined(OS_SUNOS)
@ -60,10 +58,29 @@ __attribute__((__weak__)) void checkStackSize()
SCOPE_EXIT({ pthread_attr_destroy(&attr); }); SCOPE_EXIT({ pthread_attr_destroy(&attr); });
if (0 != pthread_attr_getstack(&attr, &stack_address, &max_stack_size)) if (0 != pthread_attr_getstack(&attr, &address, &size))
throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR);
#endif // OS_DARWIN #endif // OS_DARWIN
}
if (out_address)
*out_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,
* current implementation is not reasonable, as it has no way to properly check the remaining
* stack size without knowing the details of how stacks are allocated.
* We mark this function as weak symbol to be able to replace it in another ClickHouse-based products.
*/
__attribute__((__weak__)) void checkStackSize()
{
using namespace DB;
if (!stack_address)
max_stack_size = getStackSize(&stack_address);
const void * frame_address = __builtin_frame_address(0); const void * frame_address = __builtin_frame_address(0);
uintptr_t int_frame_address = reinterpret_cast<uintptr_t>(frame_address); uintptr_t int_frame_address = reinterpret_cast<uintptr_t>(frame_address);

View File

@ -63,7 +63,7 @@ SRCS(
PipeFDs.cpp PipeFDs.cpp
ProcfsMetricsProvider.cpp ProcfsMetricsProvider.cpp
ProfileEvents.cpp ProfileEvents.cpp
ProgressBar.cpp ProgressIndication.cpp
QueryProfiler.cpp QueryProfiler.cpp
RWLock.cpp RWLock.cpp
RemoteHostFilter.cpp RemoteHostFilter.cpp

View File

@ -23,9 +23,10 @@ using IndexToLogEntry = std::unordered_map<uint64_t, LogEntryPtr>;
enum class ChangelogVersion : uint8_t enum class ChangelogVersion : uint8_t
{ {
V0 = 0, V0 = 0,
V1 = 1, /// with 64 bit buffer header
}; };
static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V0; static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V1;
struct ChangelogRecordHeader struct ChangelogRecordHeader
{ {

View File

@ -204,7 +204,7 @@ SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage,
uint8_t version; uint8_t version;
readBinary(version, in); readBinary(version, in);
SnapshotVersion current_version = static_cast<SnapshotVersion>(version); SnapshotVersion current_version = static_cast<SnapshotVersion>(version);
if (current_version > SnapshotVersion::V1) if (current_version > CURRENT_SNAPSHOT_VERSION)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version); throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version);
SnapshotMetadataPtr result = deserializeSnapshotMetadata(in); SnapshotMetadataPtr result = deserializeSnapshotMetadata(in);

View File

@ -14,8 +14,11 @@ enum SnapshotVersion : uint8_t
{ {
V0 = 0, V0 = 0,
V1 = 1, /// with ACL map V1 = 1, /// with ACL map
V2 = 2, /// with 64 bit buffer header
}; };
static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V2;
struct KeeperStorageSnapshot struct KeeperStorageSnapshot
{ {
public: public:
@ -30,7 +33,7 @@ public:
KeeperStorage * storage; KeeperStorage * storage;
SnapshotVersion version = SnapshotVersion::V1; SnapshotVersion version = CURRENT_SNAPSHOT_VERSION;
SnapshotMetadataPtr snapshot_meta; SnapshotMetadataPtr snapshot_meta;
int64_t session_id; int64_t session_id;
size_t snapshot_container_size; size_t snapshot_container_size;

View File

@ -24,6 +24,8 @@ namespace QueryProcessingStage
stage = WithMergeableState; stage = WithMergeableState;
else if (stage_string == "with_mergeable_state_after_aggregation") else if (stage_string == "with_mergeable_state_after_aggregation")
stage = WithMergeableStateAfterAggregation; stage = WithMergeableStateAfterAggregation;
else if (stage_string == "with_mergeable_state_after_aggregation_and_limit")
stage = WithMergeableStateAfterAggregationAndLimit;
else else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query processing stage: {}", stage_string); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query processing stage: {}", stage_string);

View File

@ -26,8 +26,15 @@ namespace QueryProcessingStage
/// It is used for auto distributed_group_by_no_merge optimization for distributed engine. /// It is used for auto distributed_group_by_no_merge optimization for distributed engine.
/// (See comments in StorageDistributed). /// (See comments in StorageDistributed).
WithMergeableStateAfterAggregation = 3, 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) inline const char * toString(UInt64 stage)
@ -38,6 +45,7 @@ namespace QueryProcessingStage
"WithMergeableState", "WithMergeableState",
"Complete", "Complete",
"WithMergeableStateAfterAggregation", "WithMergeableStateAfterAggregation",
"WithMergeableStateAfterAggregationAndLimit",
}; };
return stage < MAX return stage < MAX
? data[stage] ? data[stage]

View File

@ -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, 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 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(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(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) \ 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) \
@ -563,7 +564,8 @@ class IColumn;
M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ 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, 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, 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 // End of FORMAT_FACTORY_SETTINGS
// Please add settings non-related to formats into the COMMON_SETTINGS above. // Please add settings non-related to formats into the COMMON_SETTINGS above.

View File

@ -170,7 +170,7 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view
{ {
ReadBufferFromString in(value); ReadBufferFromString in(value);
time_t time = 0; time_t time = 0;
readDateTimeText(time, in); readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(data_type.get())->getTimeZone());
if (time < 0) if (time < 0)
time = 0; time = 0;
assert_cast<ColumnUInt32 &>(column).insertValue(time); assert_cast<ColumnUInt32 &>(column).insertValue(time);
@ -272,11 +272,11 @@ void PostgreSQLBlockInputStream::prepareArrayInfo(size_t column_idx, const DataT
else if (which.isDate()) else if (which.isDate())
parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; }; parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; };
else if (which.isDateTime()) else if (which.isDateTime())
parser = [](std::string & field) -> Field parser = [nested](std::string & field) -> Field
{ {
ReadBufferFromString in(field); ReadBufferFromString in(field);
time_t time = 0; time_t time = 0;
readDateTimeText(time, in); readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(nested.get())->getTimeZone());
return time; return time;
}; };
else if (which.isDecimal32()) else if (which.isDecimal32())

View File

@ -112,6 +112,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; 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.with_names_use_header = settings.input_format_with_names_use_header;
format_settings.write_statistics = settings.output_format_write_statistics; 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 /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (format_settings.schema.is_server) if (format_settings.schema.is_server)

View File

@ -52,6 +52,7 @@ struct FormatSettings
struct struct
{ {
UInt64 row_group_size = 1000000; UInt64 row_group_size = 1000000;
bool low_cardinality_as_dictionary = false;
} arrow; } arrow;
struct struct

View File

@ -169,7 +169,7 @@ namespace
{ {
ReadBufferFromString in(value); ReadBufferFromString in(value);
time_t time = 0; time_t time = 0;
readDateTimeText(time, in); readDateTimeText(time, in, assert_cast<const DataTypeDateTime &>(data_type).getTimeZone());
if (time < 0) if (time < 0)
time = 0; time = 0;
assert_cast<ColumnUInt32 &>(column).insertValue(time); assert_cast<ColumnUInt32 &>(column).insertValue(time);

View File

@ -63,6 +63,83 @@ void ProgressValues::writeJSON(WriteBuffer & out) const
writeCString("\"}", out); writeCString("\"}", out);
} }
bool Progress::incrementPiecewiseAtomically(const Progress & rhs)
{
read_rows += rhs.read_rows;
read_bytes += rhs.read_bytes;
read_raw_bytes += rhs.read_raw_bytes;
total_rows_to_read += rhs.total_rows_to_read;
total_raw_bytes_to_read += rhs.total_raw_bytes_to_read;
written_rows += rhs.written_rows;
written_bytes += rhs.written_bytes;
return rhs.read_rows || rhs.written_rows;
}
void Progress::reset()
{
read_rows = 0;
read_bytes = 0;
read_raw_bytes = 0;
total_rows_to_read = 0;
total_raw_bytes_to_read = 0;
written_rows = 0;
written_bytes = 0;
}
ProgressValues Progress::getValues() const
{
ProgressValues res;
res.read_rows = read_rows.load(std::memory_order_relaxed);
res.read_bytes = read_bytes.load(std::memory_order_relaxed);
res.read_raw_bytes = read_raw_bytes.load(std::memory_order_relaxed);
res.total_rows_to_read = total_rows_to_read.load(std::memory_order_relaxed);
res.total_raw_bytes_to_read = total_raw_bytes_to_read.load(std::memory_order_relaxed);
res.written_rows = written_rows.load(std::memory_order_relaxed);
res.written_bytes = written_bytes.load(std::memory_order_relaxed);
return res;
}
ProgressValues Progress::fetchAndResetPiecewiseAtomically()
{
ProgressValues res;
res.read_rows = read_rows.fetch_and(0);
res.read_bytes = read_bytes.fetch_and(0);
res.read_raw_bytes = read_raw_bytes.fetch_and(0);
res.total_rows_to_read = total_rows_to_read.fetch_and(0);
res.total_raw_bytes_to_read = total_raw_bytes_to_read.fetch_and(0);
res.written_rows = written_rows.fetch_and(0);
res.written_bytes = written_bytes.fetch_and(0);
return res;
}
Progress & Progress::operator=(Progress && other)
{
read_rows = other.read_rows.load(std::memory_order_relaxed);
read_bytes = other.read_bytes.load(std::memory_order_relaxed);
read_raw_bytes = other.read_raw_bytes.load(std::memory_order_relaxed);
total_rows_to_read = other.total_rows_to_read.load(std::memory_order_relaxed);
total_raw_bytes_to_read = other.total_raw_bytes_to_read.load(std::memory_order_relaxed);
written_rows = other.written_rows.load(std::memory_order_relaxed);
written_bytes = other.written_bytes.load(std::memory_order_relaxed);
return *this;
}
void Progress::read(ReadBuffer & in, UInt64 server_revision) void Progress::read(ReadBuffer & in, UInt64 server_revision)
{ {
ProgressValues values; ProgressValues values;

View File

@ -5,6 +5,8 @@
#include <common/types.h> #include <common/types.h>
#include <Core/Defines.h> #include <Core/Defines.h>
#include <Common/Stopwatch.h>
namespace DB namespace DB
{ {
@ -17,7 +19,11 @@ struct ProgressValues
{ {
size_t read_rows; size_t read_rows;
size_t read_bytes; size_t read_bytes;
size_t read_raw_bytes;
size_t total_rows_to_read; size_t total_rows_to_read;
size_t total_raw_bytes_to_read;
size_t written_rows; size_t written_rows;
size_t written_bytes; size_t written_bytes;
@ -45,6 +51,16 @@ struct WriteProgress
: written_rows(written_rows_), written_bytes(written_bytes_) {} : written_rows(written_rows_), written_bytes(written_bytes_) {}
}; };
struct FileProgress
{
/// Here read_bytes (raw bytes) - do not equal ReadProgress::read_bytes, which are calculated according to column types.
size_t read_bytes;
size_t total_bytes_to_read;
FileProgress(size_t read_bytes_, size_t total_bytes_to_read_ = 0) : read_bytes(read_bytes_), total_bytes_to_read(total_bytes_to_read_) {}
};
/** Progress of query execution. /** Progress of query execution.
* Values, transferred over network are deltas - how much was done after previously sent value. * Values, transferred over network are deltas - how much was done after previously sent value.
* The same struct is also used for summarized values. * The same struct is also used for summarized values.
@ -53,87 +69,50 @@ struct Progress
{ {
std::atomic<size_t> read_rows {0}; /// Rows (source) processed. std::atomic<size_t> read_rows {0}; /// Rows (source) processed.
std::atomic<size_t> read_bytes {0}; /// Bytes (uncompressed, source) processed. std::atomic<size_t> read_bytes {0}; /// Bytes (uncompressed, source) processed.
std::atomic<size_t> read_raw_bytes {0}; /// Raw bytes processed.
/** How much rows must be processed, in total, approximately. Non-zero value is sent when there is information about some new part of job. /** How much rows/bytes must be processed, in total, approximately. Non-zero value is sent when there is information about
* Received values must be summed to get estimate of total rows to process. * some new part of job. Received values must be summed to get estimate of total rows to process.
* `total_raw_bytes_to_process` is used for file table engine or when reading from file descriptor.
* Used for rendering progress bar on client. * Used for rendering progress bar on client.
*/ */
std::atomic<size_t> total_rows_to_read {0}; std::atomic<size_t> total_rows_to_read {0};
std::atomic<size_t> total_raw_bytes_to_read {0};
std::atomic<size_t> written_rows {0}; std::atomic<size_t> written_rows {0};
std::atomic<size_t> written_bytes {0}; std::atomic<size_t> written_bytes {0};
Progress() = default; Progress() = default;
Progress(size_t read_rows_, size_t read_bytes_, size_t total_rows_to_read_ = 0) Progress(size_t read_rows_, size_t read_bytes_, size_t total_rows_to_read_ = 0)
: read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_) {} : read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_) {}
explicit Progress(ReadProgress read_progress) explicit Progress(ReadProgress read_progress)
: read_rows(read_progress.read_rows), read_bytes(read_progress.read_bytes), total_rows_to_read(read_progress.total_rows_to_read) {} : read_rows(read_progress.read_rows), read_bytes(read_progress.read_bytes), total_rows_to_read(read_progress.total_rows_to_read) {}
explicit Progress(WriteProgress write_progress) explicit Progress(WriteProgress write_progress)
: written_rows(write_progress.written_rows), written_bytes(write_progress.written_bytes) {} : written_rows(write_progress.written_rows), written_bytes(write_progress.written_bytes) {}
explicit Progress(FileProgress file_progress)
: read_raw_bytes(file_progress.read_bytes), total_raw_bytes_to_read(file_progress.total_bytes_to_read) {}
void read(ReadBuffer & in, UInt64 server_revision); void read(ReadBuffer & in, UInt64 server_revision);
void write(WriteBuffer & out, UInt64 client_revision) const; void write(WriteBuffer & out, UInt64 client_revision) const;
/// Progress in JSON format (single line, without whitespaces) is used in HTTP headers. /// Progress in JSON format (single line, without whitespaces) is used in HTTP headers.
void writeJSON(WriteBuffer & out) const; void writeJSON(WriteBuffer & out) const;
/// Each value separately is changed atomically (but not whole object). /// Each value separately is changed atomically (but not whole object).
bool incrementPiecewiseAtomically(const Progress & rhs) bool incrementPiecewiseAtomically(const Progress & rhs);
{
read_rows += rhs.read_rows;
read_bytes += rhs.read_bytes;
total_rows_to_read += rhs.total_rows_to_read;
written_rows += rhs.written_rows;
written_bytes += rhs.written_bytes;
return rhs.read_rows || rhs.written_rows; void reset();
}
void reset() ProgressValues getValues() const;
{
read_rows = 0;
read_bytes = 0;
total_rows_to_read = 0;
written_rows = 0;
written_bytes = 0;
}
ProgressValues getValues() const ProgressValues fetchAndResetPiecewiseAtomically();
{
ProgressValues res;
res.read_rows = read_rows.load(std::memory_order_relaxed); Progress & operator=(Progress && other);
res.read_bytes = read_bytes.load(std::memory_order_relaxed);
res.total_rows_to_read = total_rows_to_read.load(std::memory_order_relaxed);
res.written_rows = written_rows.load(std::memory_order_relaxed);
res.written_bytes = written_bytes.load(std::memory_order_relaxed);
return res;
}
ProgressValues fetchAndResetPiecewiseAtomically()
{
ProgressValues res;
res.read_rows = read_rows.fetch_and(0);
res.read_bytes = read_bytes.fetch_and(0);
res.total_rows_to_read = total_rows_to_read.fetch_and(0);
res.written_rows = written_rows.fetch_and(0);
res.written_bytes = written_bytes.fetch_and(0);
return res;
}
Progress & operator=(Progress && other)
{
read_rows = other.read_rows.load(std::memory_order_relaxed);
read_bytes = other.read_bytes.load(std::memory_order_relaxed);
total_rows_to_read = other.total_rows_to_read.load(std::memory_order_relaxed);
written_rows = other.written_rows.load(std::memory_order_relaxed);
written_bytes = other.written_bytes.load(std::memory_order_relaxed);
return *this;
}
Progress(Progress && other) Progress(Progress && other)
{ {

View File

@ -6,7 +6,12 @@
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <IO/ReadBufferFromFileDescriptor.h> #include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <sys/stat.h>
#include <Common/UnicodeBar.h>
#include <Common/TerminalSize.h>
#include <IO/Operators.h>
namespace ProfileEvents namespace ProfileEvents
@ -32,6 +37,7 @@ namespace ErrorCodes
extern const int ARGUMENT_OUT_OF_BOUND; extern const int ARGUMENT_OUT_OF_BOUND;
extern const int CANNOT_SEEK_THROUGH_FILE; extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int CANNOT_SELECT; extern const int CANNOT_SELECT;
extern const int CANNOT_FSTAT;
} }
@ -170,4 +176,28 @@ bool ReadBufferFromFileDescriptor::poll(size_t timeout_microseconds)
return res > 0; return res > 0;
} }
off_t ReadBufferFromFileDescriptor::size()
{
struct stat buf;
int res = fstat(fd, &buf);
if (-1 == res)
throwFromErrnoWithPath("Cannot execute fstat " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSTAT);
return buf.st_size;
}
void ReadBufferFromFileDescriptor::setProgressCallback(ContextPtr context)
{
auto file_progress_callback = context->getFileProgressCallback();
if (!file_progress_callback)
return;
setProfileCallback([file_progress_callback](const ProfileInfo & progress)
{
file_progress_callback(FileProgress(progress.bytes_read, 0));
});
}
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <IO/ReadBufferFromFileBase.h> #include <IO/ReadBufferFromFileBase.h>
#include <Interpreters/Context.h>
#include <unistd.h> #include <unistd.h>
@ -38,6 +39,9 @@ public:
/// If 'offset' is small enough to stay in buffer after seek, then true seek in file does not happen. /// If 'offset' is small enough to stay in buffer after seek, then true seek in file does not happen.
off_t seek(off_t off, int whence) override; off_t seek(off_t off, int whence) override;
off_t size();
void setProgressCallback(ContextPtr context);
private: private:
/// Assuming file descriptor supports 'select', check that we have data to read or wait until timeout. /// Assuming file descriptor supports 'select', check that we have data to read or wait until timeout.
bool poll(size_t timeout_microseconds); bool poll(size_t timeout_microseconds);

View File

@ -181,8 +181,13 @@ private:
std::shared_ptr<const EnabledRowPolicies> initial_row_policy; std::shared_ptr<const EnabledRowPolicies> initial_row_policy;
String current_database; String current_database;
Settings settings; /// Setting for query execution. Settings settings; /// Setting for query execution.
using ProgressCallback = std::function<void(const Progress & progress)>; using ProgressCallback = std::function<void(const Progress & progress)>;
ProgressCallback progress_callback; /// Callback for tracking progress of query execution. ProgressCallback progress_callback; /// Callback for tracking progress of query execution.
using FileProgressCallback = std::function<void(const FileProgress & progress)>;
FileProgressCallback file_progress_callback; /// Callback for tracking progress of file loading.
QueryStatus * process_list_elem = nullptr; /// For tracking total resource usage for query. QueryStatus * process_list_elem = nullptr; /// For tracking total resource usage for query.
StorageID insertion_table = StorageID::createEmpty(); /// Saved insertion table in query context StorageID insertion_table = StorageID::createEmpty(); /// Saved insertion table in query context
@ -588,6 +593,9 @@ public:
/// Used in InterpreterSelectQuery to pass it to the IBlockInputStream. /// Used in InterpreterSelectQuery to pass it to the IBlockInputStream.
ProgressCallback getProgressCallback() const; ProgressCallback getProgressCallback() const;
void setFileProgressCallback(FileProgressCallback && callback) { file_progress_callback = callback; }
FileProgressCallback getFileProgressCallback() const { return file_progress_callback; }
/** Set in executeQuery and InterpreterSelectQuery. Then it is used in IBlockInputStream, /** Set in executeQuery and InterpreterSelectQuery. Then it is used in IBlockInputStream,
* to update and monitor information about the total number of resources spent for the query. * to update and monitor information about the total number of resources spent for the query.
*/ */

View File

@ -831,14 +831,17 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (create.attach && !create.storage && !create.columns_list) if (create.attach && !create.storage && !create.columns_list)
{ {
auto database = DatabaseCatalog::instance().getDatabase(database_name); auto database = DatabaseCatalog::instance().getDatabase(database_name);
if (database->getEngineName() == "Replicated") if (database->getEngineName() == "Replicated")
{ {
auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.table); auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.table);
if (typeid_cast<DatabaseReplicated *>(database.get()) && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
if (auto* ptr = typeid_cast<DatabaseReplicated *>(database.get());
ptr && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{ {
create.database = database_name; create.database = database_name;
guard->releaseTableLock(); guard->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(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") if (need_add_to_database && database->getEngineName() == "Replicated")
{ {
auto guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table); auto guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table);
if (typeid_cast<DatabaseReplicated *>(database.get()) && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
if (auto * ptr = typeid_cast<DatabaseReplicated *>(database.get());
ptr && getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{ {
assertOrSetUUID(create, database); assertOrSetUUID(create, database);
guard->releaseTableLock(); guard->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(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); data_path = database->getTableDataPath(create);
if (!create.attach && !data_path.empty() && fs::exists(fs::path{getContext()->getPath()} / data_path)) 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 else
{ {

View File

@ -189,12 +189,11 @@ BlockIO InterpreterInsertQuery::execute()
const auto & union_modes = select_query.list_of_modes; 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 /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries
is_trivial_insert_select const auto mode_is_all = [](const auto & mode) { return mode == ASTSelectWithUnionQuery::Mode::ALL; };
= std::all_of(
union_modes.begin(), is_trivial_insert_select =
union_modes.end(), std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all))
[](const ASTSelectWithUnionQuery::Mode & mode) { return mode == ASTSelectWithUnionQuery::Mode::ALL; }) && std::all_of(selects.begin(), selects.end(), isTrivialSelect);
&& std::all_of(selects.begin(), selects.end(), [](const ASTPtr & select) { return isTrivialSelect(select); });
} }
if (is_trivial_insert_select) if (is_trivial_insert_select)

View File

@ -633,7 +633,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
/// Running on the initiating server during distributed processing or if query is not distributed. /// 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) /// 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. /// So it will do partial second stage (second_stage=true), and initiator will do the final part.
bool second_stage = from_stage <= QueryProcessingStage::WithMergeableState bool second_stage = from_stage <= QueryProcessingStage::WithMergeableState
&& options.to_stage > QueryProcessingStage::WithMergeableState; && options.to_stage > QueryProcessingStage::WithMergeableState;
@ -705,7 +705,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
return res; 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 // It's different from selected_columns, see the comment above for
// WithMergeableState stage. // WithMergeableState stage.
@ -1012,10 +1012,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
/// Support optimize_distributed_group_by_sharding_key /// Support optimize_distributed_group_by_sharding_key
/// Is running on the initiating server during distributed processing? /// Is running on the initiating server during distributed processing?
if (from_stage == QueryProcessingStage::WithMergeableStateAfterAggregation) if (from_stage >= QueryProcessingStage::WithMergeableStateAfterAggregation)
from_aggregation_stage = true; from_aggregation_stage = true;
/// Is running on remote servers during distributed processing? /// Is running on remote servers during distributed processing?
if (options.to_stage == QueryProcessingStage::WithMergeableStateAfterAggregation) if (options.to_stage >= QueryProcessingStage::WithMergeableStateAfterAggregation)
to_aggregation_stage = true; to_aggregation_stage = true;
/// Read the data from Storage. from_stage - to what stage the request was completed in Storage. /// Read the data from Storage. from_stage - to what stage the request was completed in Storage.
@ -1301,7 +1301,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
* but there is no aggregation, then on the remote servers ORDER BY was made * but there is no aggregation, then on the remote servers ORDER BY was made
* - therefore, we merge the sorted streams from remote servers. * - 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), * (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. * then merge the sorted streams is enough, since remote servers already did full ORDER BY.
*/ */
@ -1335,13 +1335,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; bool has_prelimit = false;
if (!to_aggregation_stage && if (apply_limit &&
query.limitLength() && !query.limit_with_ties && !hasWithTotalsInAnySubqueryInFromClause(query) && query.limitLength() && !query.limit_with_ties && !hasWithTotalsInAnySubqueryInFromClause(query) &&
!query.arrayJoinExpressionList() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes && !query.arrayJoinExpressionList() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes &&
!has_withfill) !has_withfill)
{ {
executePreLimit(query_plan, false); executePreLimit(query_plan, /* do_not_skip_offset= */!apply_offset);
has_prelimit = true; has_prelimit = true;
} }
@ -1368,7 +1370,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. /// 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) if (!to_aggregation_stage)
{ {
/// We must do projection after DISTINCT because projection may remove some columns. /// We must do projection after DISTINCT because projection may remove some columns.
@ -1379,10 +1381,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
executeExtremes(query_plan); executeExtremes(query_plan);
/// Limit is no longer needed if there is prelimit. /// Limit is no longer needed if there is prelimit.
if (!to_aggregation_stage && !has_prelimit) if (apply_limit && !has_prelimit)
executeLimit(query_plan); executeLimit(query_plan);
if (!to_aggregation_stage) if (apply_offset)
executeOffset(query_plan); executeOffset(query_plan);
} }
} }

View File

@ -43,11 +43,8 @@
# include "config_core.h" # include "config_core.h"
#endif #endif
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
@ -56,6 +53,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED; extern const int NOT_IMPLEMENTED;
extern const int TIMEOUT_EXCEEDED; extern const int TIMEOUT_EXCEEDED;
extern const int TABLE_WAS_NOT_DROPPED; 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); 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] <something action from ActionLocks> /// Implements SYSTEM [START|STOP] <something action from ActionLocks>
@ -212,11 +212,16 @@ BlockIO InterpreterSystemQuery::execute()
system_context->setSetting("profile", getContext()->getSystemProfileName()); system_context->setSetting("profile", getContext()->getSystemProfileName());
/// Make canonical query for simpler processing /// 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); 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 = {}; volume_ptr = {};
if (!query.storage_policy.empty() && !query.volume.empty()) if (!query.storage_policy.empty() && !query.volume.empty())
@ -286,7 +291,7 @@ BlockIO InterpreterSystemQuery::execute()
getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY);
auto & external_dictionaries_loader = system_context->getExternalDictionariesLoader(); auto & external_dictionaries_loader = system_context->getExternalDictionariesLoader();
external_dictionaries_loader.reloadDictionary(query.target_dictionary, getContext()); external_dictionaries_loader.reloadDictionary(query.table, getContext());
ExternalDictionariesLoader::resetAll(); ExternalDictionariesLoader::resetAll();
@ -296,8 +301,8 @@ BlockIO InterpreterSystemQuery::execute()
{ {
getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY); getContext()->checkAccess(AccessType::SYSTEM_RELOAD_DICTIONARY);
executeCommandsAndThrowIfError( executeCommandsAndThrowIfError(
[&] () { system_context->getExternalDictionariesLoader().reloadAllTriedToLoad(); }, [&] { system_context->getExternalDictionariesLoader().reloadAllTriedToLoad(); },
[&] () { system_context->getEmbeddedDictionaries().reload(); } [&] { system_context->getEmbeddedDictionaries().reload(); }
); );
ExternalDictionariesLoader::resetAll(); ExternalDictionariesLoader::resetAll();
break; break;
@ -392,8 +397,10 @@ BlockIO InterpreterSystemQuery::execute()
break; break;
case Type::RESTART_REPLICA: case Type::RESTART_REPLICA:
if (!tryRestartReplica(table_id, system_context)) if (!tryRestartReplica(table_id, system_context))
throw Exception("There is no " + query.database + "." + query.table + " replicated table", throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs());
ErrorCodes::BAD_ARGUMENTS); break;
case Type::RESTORE_REPLICA:
restoreReplica();
break; break;
case Type::RESTART_DISK: case Type::RESTART_DISK:
restartDisk(query.disk); restartDisk(query.disk);
@ -402,14 +409,14 @@ BlockIO InterpreterSystemQuery::execute()
{ {
getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS); getContext()->checkAccess(AccessType::SYSTEM_FLUSH_LOGS);
executeCommandsAndThrowIfError( executeCommandsAndThrowIfError(
[&] () { if (auto query_log = getContext()->getQueryLog()) query_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 part_log = getContext()->getPartLog("")) part_log->flush(true); },
[&] () { if (auto query_thread_log = getContext()->getQueryThreadLog()) query_thread_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 trace_log = getContext()->getTraceLog()) trace_log->flush(true); },
[&] () { if (auto text_log = getContext()->getTextLog()) text_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 metric_log = getContext()->getMetricLog()) metric_log->flush(true); },
[&] () { if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) asynchronous_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 opentelemetry_span_log = getContext()->getOpenTelemetrySpanLog()) opentelemetry_span_log->flush(true); }
); );
break; break;
} }
@ -423,12 +430,51 @@ BlockIO InterpreterSystemQuery::execute()
return BlockIO(); 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<StorageReplicatedMergeTree *>(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) StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard)
{ {
getContext()->checkAccess(AccessType::SYSTEM_RESTART_REPLICA, replica); 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()); auto [database, table] = DatabaseCatalog::instance().tryGetDatabaseAndTable(replica, getContext());
ASTPtr create_ast; ASTPtr create_ast;
@ -475,28 +521,23 @@ void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context)
auto & catalog = DatabaseCatalog::instance(); auto & catalog = DatabaseCatalog::instance();
for (auto & elem : catalog.getDatabases()) for (auto & elem : catalog.getDatabases())
{ for (auto it = elem.second->getTablesIterator(getContext()); it->isValid(); it->next())
DatabasePtr & database = elem.second; if (dynamic_cast<const StorageReplicatedMergeTree *>(it->table().get()))
for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) replica_names.emplace_back(it->databaseName(), it->name());
{
if (auto table = iterator->table())
{
if (dynamic_cast<const StorageReplicatedMergeTree *>(table.get()))
replica_names.emplace_back(StorageID{iterator->databaseName(), iterator->name()});
}
}
}
if (replica_names.empty()) if (replica_names.empty())
return; return;
TableGuards guards; TableGuards guards;
for (const auto & name : replica_names) for (const auto & name : replica_names)
guards.emplace(UniqueTableName{name.database_name, name.table_name}, nullptr); guards.emplace(UniqueTableName{name.database_name, name.table_name}, nullptr);
for (auto & guard : guards) for (auto & guard : guards)
guard.second = catalog.getDDLGuard(guard.first.database_name, guard.first.table_name); guard.second = catalog.getDDLGuard(guard.first.database_name, guard.first.table_name);
ThreadPool pool(std::min(size_t(getNumberOfPhysicalCPUCores()), replica_names.size())); ThreadPool pool(std::min(size_t(getNumberOfPhysicalCPUCores()), replica_names.size()));
for (auto & replica : replica_names) for (auto & replica : replica_names)
{ {
LOG_TRACE(log, "Restarting replica on {}", replica.getNameForLogs()); 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()); StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
if (!dropReplicaImpl(query, table)) 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()) else if (!query.database.empty())
{ {
@ -628,7 +669,7 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery &)
LOG_TRACE(log, "SYNC REPLICA {}: OK", table_id.getNameForLogs()); LOG_TRACE(log, "SYNC REPLICA {}: OK", table_id.getNameForLogs());
} }
else 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 &) void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &)
@ -659,6 +700,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
const auto & query = query_ptr->as<const ASTSystemQuery &>(); const auto & query = query_ptr->as<const ASTSystemQuery &>();
using Type = ASTSystemQuery::Type; using Type = ASTSystemQuery::Type;
AccessRightsElements required_access; AccessRightsElements required_access;
switch (query.type) switch (query.type)
{ {
case Type::SHUTDOWN: [[fallthrough]]; case Type::SHUTDOWN: [[fallthrough]];
@ -770,6 +812,11 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
required_access.emplace_back(AccessType::SYSTEM_DROP_REPLICA, query.database, query.table); required_access.emplace_back(AccessType::SYSTEM_DROP_REPLICA, query.database, query.table);
break; break;
} }
case Type::RESTORE_REPLICA:
{
required_access.emplace_back(AccessType::SYSTEM_RESTORE_REPLICA, query.database, query.table);
break;
}
case Type::SYNC_REPLICA: case Type::SYNC_REPLICA:
{ {
required_access.emplace_back(AccessType::SYSTEM_SYNC_REPLICA, query.database, query.table); required_access.emplace_back(AccessType::SYSTEM_SYNC_REPLICA, query.database, query.table);

View File

@ -49,6 +49,9 @@ private:
void restartReplicas(ContextMutablePtr system_context); void restartReplicas(ContextMutablePtr system_context);
void syncReplica(ASTSystemQuery & query); void syncReplica(ASTSystemQuery & query);
void restoreReplica();
void dropReplica(ASTSystemQuery & query); void dropReplica(ASTSystemQuery & query);
bool dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table); bool dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table);
void flushDistributed(ASTSystemQuery & query); void flushDistributed(ASTSystemQuery & query);

View File

@ -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 current database in a distributed query need to be replaced with either
/// the local current database or a shard's default database. /// the local current database or a shard's default database.
bool need_replace_current_database bool need_replace_current_database = std::any_of(
= (std::find_if(
query_requires_access.begin(), query_requires_access.begin(),
query_requires_access.end(), query_requires_access.end(),
[](const AccessRightsElement & elem) { return elem.isEmptyDatabase(); }) [](const AccessRightsElement & elem) { return elem.isEmptyDatabase(); });
!= query_requires_access.end());
bool use_local_default_database = false; bool use_local_default_database = false;
const String & current_database = context->getCurrentDatabase(); const String & current_database = context->getCurrentDatabase();

View File

@ -113,6 +113,7 @@ Block getHeaderForProcessingStage(
case QueryProcessingStage::WithMergeableState: case QueryProcessingStage::WithMergeableState:
case QueryProcessingStage::Complete: case QueryProcessingStage::Complete:
case QueryProcessingStage::WithMergeableStateAfterAggregation: case QueryProcessingStage::WithMergeableStateAfterAggregation:
case QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit:
case QueryProcessingStage::MAX: case QueryProcessingStage::MAX:
{ {
auto query = query_info.query->clone(); auto query = query_info.query->clone();

View File

@ -130,9 +130,14 @@ void removeColumnNullability(ColumnWithTypeAndName & column)
const auto & dict_type = typeid_cast<const DataTypeLowCardinality *>(column.type.get())->getDictionaryType(); const auto & dict_type = typeid_cast<const DataTypeLowCardinality *>(column.type.get())->getDictionaryType();
column.type = std::make_shared<DataTypeLowCardinality>(removeNullable(dict_type)); column.type = std::make_shared<DataTypeLowCardinality>(removeNullable(dict_type));
ColumnLowCardinality * col_as_lc = typeid_cast<ColumnLowCardinality *>(column.column->assumeMutable().get()); if (column.column && column.column->lowCardinality())
{
auto mut_col = IColumn::mutate(std::move(column.column));
ColumnLowCardinality * col_as_lc = typeid_cast<ColumnLowCardinality *>(mut_col.get());
if (col_as_lc && col_as_lc->nestedIsNullable()) if (col_as_lc && col_as_lc->nestedIsNullable())
col_as_lc->nestedRemoveNullable(); col_as_lc->nestedRemoveNullable();
column.column = std::move(mut_col);
}
return; return;
} }

View File

@ -44,6 +44,8 @@ const char * ASTSystemQuery::typeToString(Type type)
return "RESTART REPLICAS"; return "RESTART REPLICAS";
case Type::RESTART_REPLICA: case Type::RESTART_REPLICA:
return "RESTART REPLICA"; return "RESTART REPLICA";
case Type::RESTORE_REPLICA:
return "RESTORE REPLICA";
case Type::DROP_REPLICA: case Type::DROP_REPLICA:
return "DROP REPLICA"; return "DROP REPLICA";
case Type::SYNC_REPLICA: case Type::SYNC_REPLICA:
@ -119,18 +121,6 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
<< (settings.hilite ? hilite_none : ""); << (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 = [&] auto print_drop_replica = [&]
{ {
settings.ostr << " " << quoteString(replica); settings.ostr << " " << quoteString(replica);
@ -187,14 +177,14 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
else if (!volume.empty()) else if (!volume.empty())
print_on_volume(); 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(); print_database_table();
} }
else if (type == Type::RELOAD_DICTIONARY)
{
print_database_dictionary();
}
else if (type == Type::DROP_REPLICA) else if (type == Type::DROP_REPLICA)
{ {
print_drop_replica(); print_drop_replica();

View File

@ -32,6 +32,7 @@ public:
START_LISTEN_QUERIES, START_LISTEN_QUERIES,
RESTART_REPLICAS, RESTART_REPLICAS,
RESTART_REPLICA, RESTART_REPLICA,
RESTORE_REPLICA,
DROP_REPLICA, DROP_REPLICA,
SYNC_REPLICA, SYNC_REPLICA,
RELOAD_DICTIONARY, RELOAD_DICTIONARY,
@ -65,7 +66,6 @@ public:
Type type = Type::UNKNOWN; Type type = Type::UNKNOWN;
String target_dictionary;
String target_model; String target_model;
String database; String database;
String table; String table;

View File

@ -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. /** 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. * 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)) if (it[0] && 0 == strncmp(it[0], "NOT", 3))
{ {
/// Was there an even number of NOTs.
bool even = false;
const char ** jt;
while (true) while (true)
{ {
const char ** jt;
for (jt = operators; *jt; jt += 2) for (jt = operators; *jt; jt += 2)
if (parseOperator(pos, *jt, expected)) if (parseOperator(pos, *jt, expected))
break; break;
@ -504,11 +502,8 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex
if (!*jt) if (!*jt)
break; 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; ASTPtr elem;
@ -518,6 +513,8 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex
if (!*it) if (!*it)
node = elem; node = elem;
else else
{
for (size_t i = 0; i < count; ++i)
{ {
/// the function corresponding to the operator /// the function corresponding to the operator
auto function = std::make_shared<ASTFunction>(); auto function = std::make_shared<ASTFunction>();
@ -529,10 +526,14 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex
function->arguments = exp_list; function->arguments = exp_list;
function->children.push_back(exp_list); function->children.push_back(exp_list);
if (node)
exp_list->children.push_back(node);
else
exp_list->children.push_back(elem); exp_list->children.push_back(elem);
node = function; node = function;
} }
}
return true; return true;
} }

View File

@ -133,7 +133,7 @@ ASTPtr SystemQuery::convertToOld() const
{ {
auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld()); auto table = std::static_pointer_cast<ASTTableIdentifier>(get(TABLE)->convertToOld());
query->database = table->getDatabaseName(); query->database = table->getDatabaseName();
query->target_dictionary = table->shortName(); query->table = table->getTableId().table_name;
} }
break; break;
case QueryType::REPLICATED_SENDS: case QueryType::REPLICATED_SENDS:

View File

@ -95,21 +95,18 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ASTRenameQuery::Elements elements; ASTRenameQuery::Elements elements;
auto ignore_delim = [&]() const auto ignore_delim = [&] { return exchange ? s_and.ignore(pos) : s_to.ignore(pos); };
{
return exchange ? s_and.ignore(pos) : s_to.ignore(pos);
};
while (true) while (true)
{ {
if (!elements.empty() && !s_comma.ignore(pos)) if (!elements.empty() && !s_comma.ignore(pos))
break; 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() || !ignore_delim()
|| !parseDatabaseAndTable(elements.back().to, pos, expected)) || !parseDatabaseAndTable(ref.to, pos, expected))
return false; return false;
} }

View File

@ -97,9 +97,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
} }
} }
bool has_distinct_on = false; /// SELECT [DISTINCT ON expr] [ALL/DISTINCT] [TOP N [WITH TIES]] expr list
/// SELECT [ALL/DISTINCT] [TOP N [WITH TIES]] expr list
{ {
bool has_all = false; bool has_all = false;
if (!s_select.ignore(pos, expected)) if (!s_select.ignore(pos, expected))
@ -108,8 +106,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (s_all.ignore(pos, expected)) if (s_all.ignore(pos, expected))
has_all = true; has_all = true;
if (s_distinct_on.ignore(pos, expected)) { if (s_distinct_on.ignore(pos, expected))
has_distinct_on = true; {
if (!exp_list.parse(pos, limit_by_expression_list, expected)) if (!exp_list.parse(pos, limit_by_expression_list, expected))
return false; return false;
limit_by_length = std::make_shared<ASTLiteral>(Field{UInt8(1)}); limit_by_length = std::make_shared<ASTLiteral>(Field{UInt8(1)});
@ -276,8 +274,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (limit_with_ties_occured) if (limit_with_ties_occured)
throw Exception("Can not use WITH TIES alongside LIMIT BY", ErrorCodes::LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED); throw Exception("Can not use WITH TIES alongside LIMIT BY", ErrorCodes::LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED);
if (has_distinct_on) if (limit_by_length)
throw Exception("Can not use distinct on alongside LIMIT BY", ErrorCodes::DISTINCT_ON_AND_LIMIT_BY_TOGETHER); throw Exception("Can not use DISTINCT ON alongside LIMIT BY", ErrorCodes::DISTINCT_ON_AND_LIMIT_BY_TOGETHER);
limit_by_length = limit_length; limit_by_length = limit_length;
limit_by_offset = limit_offset; limit_by_offset = limit_offset;

View File

@ -15,6 +15,47 @@ namespace ErrorCodes
namespace DB namespace DB
{ {
static bool parseQueryWithOnClusterAndMaybeTable(std::shared_ptr<ASTSystemQuery> & res, IParser::Pos & pos,
Expected & expected, bool require_table, bool allow_string_literal)
{
/// Better form for user: SYSTEM <ACTION> table ON CLUSTER cluster
/// Query rewritten form + form while executing on cluster: SYSTEM <ACTION> 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<ASTLiteral &>().value.safeGet<String>();
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) 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: case Type::RELOAD_DICTIONARY:
{ {
String cluster_str; if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ true, /* allow_string_literal = */ true))
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<ASTLiteral &>().value.safeGet<String>();
else if (!parseDatabaseAndTableName(pos, expected, res->database, res->target_dictionary))
return false; return false;
break; break;
} }
@ -145,24 +176,21 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
break; break;
} }
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 /// FLUSH DISTRIBUTED requires table
/// START/STOP DISTRIBUTED SENDS does not require table /// START/STOP DISTRIBUTED SENDS does not require table
if (res->type == Type::FLUSH_DISTRIBUTED) case Type::STOP_DISTRIBUTED_SENDS:
case Type::START_DISTRIBUTED_SENDS:
{
if (!parseQueryWithOnClusterAndMaybeTable(res, pos, expected, /* require table = */ false, /* allow_string_literal = */ false))
return 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; break;
} }

View File

@ -1,4 +1,5 @@
#include "ArrowBlockInputFormat.h" #include "ArrowBlockInputFormat.h"
#if USE_ARROW #if USE_ARROW
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
@ -29,7 +30,6 @@ ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & hea
Chunk ArrowBlockInputFormat::generate() Chunk ArrowBlockInputFormat::generate()
{ {
Chunk res; Chunk res;
const Block & header = getPort().getHeader();
arrow::Result<std::shared_ptr<arrow::RecordBatch>> batch_result; arrow::Result<std::shared_ptr<arrow::RecordBatch>> batch_result;
if (stream) if (stream)
@ -63,7 +63,7 @@ Chunk ArrowBlockInputFormat::generate()
++record_batch_current; ++record_batch_current;
ArrowColumnToCHColumn::arrowTableToCHChunk(res, *table_result, header, "Arrow"); arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result);
return res; return res;
} }
@ -81,6 +81,8 @@ void ArrowBlockInputFormat::resetParser()
void ArrowBlockInputFormat::prepareReader() void ArrowBlockInputFormat::prepareReader()
{ {
std::shared_ptr<arrow::Schema> schema;
if (stream) if (stream)
{ {
auto stream_reader_status = arrow::ipc::RecordBatchStreamReader::Open(std::make_unique<ArrowInputStreamFromReadBuffer>(in)); auto stream_reader_status = arrow::ipc::RecordBatchStreamReader::Open(std::make_unique<ArrowInputStreamFromReadBuffer>(in));
@ -88,6 +90,7 @@ void ArrowBlockInputFormat::prepareReader()
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, throw Exception(ErrorCodes::UNKNOWN_EXCEPTION,
"Error while opening a table: {}", stream_reader_status.status().ToString()); "Error while opening a table: {}", stream_reader_status.status().ToString());
stream_reader = *stream_reader_status; stream_reader = *stream_reader_status;
schema = stream_reader->schema();
} }
else else
{ {
@ -96,8 +99,11 @@ void ArrowBlockInputFormat::prepareReader()
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, throw Exception(ErrorCodes::UNKNOWN_EXCEPTION,
"Error while opening a table: {}", file_reader_status.status().ToString()); "Error while opening a table: {}", file_reader_status.status().ToString());
file_reader = *file_reader_status; file_reader = *file_reader_status;
schema = file_reader->schema();
} }
arrow_column_to_ch_column = std::make_unique<ArrowColumnToCHColumn>(getPort().getHeader(), std::move(schema), "Arrow");
if (stream) if (stream)
record_batch_total = -1; record_batch_total = -1;
else else

View File

@ -11,6 +11,7 @@ namespace DB
{ {
class ReadBuffer; class ReadBuffer;
class ArrowColumnToCHColumn;
class ArrowBlockInputFormat : public IInputFormat class ArrowBlockInputFormat : public IInputFormat
{ {
@ -32,6 +33,8 @@ private:
// The following fields are used only for Arrow format // The following fields are used only for Arrow format
std::shared_ptr<arrow::ipc::RecordBatchFileReader> file_reader; std::shared_ptr<arrow::ipc::RecordBatchFileReader> file_reader;
std::unique_ptr<ArrowColumnToCHColumn> arrow_column_to_ch_column;
int record_batch_total = 0; int record_batch_total = 0;
int record_batch_current = 0; int record_batch_current = 0;

View File

@ -18,17 +18,26 @@ namespace ErrorCodes
} }
ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_) 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<ArrowBufferedOutputStream>(out_)} : IOutputFormat(header_, out_)
, stream{stream_}
, format_settings{format_settings_}
, arrow_ostream{std::make_shared<ArrowBufferedOutputStream>(out_)}
{ {
} }
void ArrowBlockOutputFormat::consume(Chunk chunk) void ArrowBlockOutputFormat::consume(Chunk chunk)
{ {
const Block & header = getPort(PortKind::Main).getHeader();
const size_t columns_num = chunk.getNumColumns(); const size_t columns_num = chunk.getNumColumns();
std::shared_ptr<arrow::Table> arrow_table; std::shared_ptr<arrow::Table> arrow_table;
CHColumnToArrowColumn::chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow"); if (!ch_column_to_arrow_column)
{
const Block & header = getPort(PortKind::Main).getHeader();
ch_column_to_arrow_column
= std::make_unique<CHColumnToArrowColumn>(header, "Arrow", format_settings.arrow.low_cardinality_as_dictionary);
}
ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num);
if (!writer) if (!writer)
prepareWriter(arrow_table->schema()); prepareWriter(arrow_table->schema());

View File

@ -12,6 +12,8 @@ namespace arrow::ipc { class RecordBatchWriter; }
namespace DB namespace DB
{ {
class CHColumnToArrowColumn;
class ArrowBlockOutputFormat : public IOutputFormat class ArrowBlockOutputFormat : public IOutputFormat
{ {
public: public:
@ -28,6 +30,7 @@ private:
const FormatSettings format_settings; const FormatSettings format_settings;
std::shared_ptr<ArrowBufferedOutputStream> arrow_ostream; std::shared_ptr<ArrowBufferedOutputStream> arrow_ostream;
std::shared_ptr<arrow::ipc::RecordBatchWriter> writer; std::shared_ptr<arrow::ipc::RecordBatchWriter> writer;
std::unique_ptr<CHColumnToArrowColumn> ch_column_to_arrow_column;
void prepareWriter(const std::shared_ptr<arrow::Schema> & schema); void prepareWriter(const std::shared_ptr<arrow::Schema> & schema);
}; };

View File

@ -7,15 +7,22 @@
#include <DataTypes/DataTypesDecimal.h> #include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <common/DateLUTImpl.h> #include <common/DateLUTImpl.h>
#include <common/types.h> #include <common/types.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>
#include <Columns/ColumnArray.h> #include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnUnique.h>
#include <Columns/ColumnMap.h>
#include <Interpreters/castColumn.h> #include <Interpreters/castColumn.h>
#include <algorithm> #include <algorithm>
#include <DataTypes/DataTypeLowCardinality.h> #include <fmt/format.h>
namespace DB namespace DB
@ -27,6 +34,7 @@ namespace DB
extern const int CANNOT_CONVERT_TYPE; extern const int CANNOT_CONVERT_TYPE;
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
extern const int THERE_IS_NO_COLUMN; extern const int THERE_IS_NO_COLUMN;
extern const int BAD_ARGUMENTS;
} }
static const std::initializer_list<std::pair<arrow::Type::type, const char *>> arrow_type_to_internal_type = static const std::initializer_list<std::pair<arrow::Type::type, const char *>> arrow_type_to_internal_type =
@ -152,11 +160,11 @@ namespace DB
if (days_num > DATE_LUT_MAX_DAY_NUM) if (days_num > DATE_LUT_MAX_DAY_NUM)
{ {
// TODO: will it rollback correctly? // TODO: will it rollback correctly?
throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + internal_column.getName() throw Exception
+ "\" is greater than " {
"max allowed Date value, which is " 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),
+ std::to_string(DATE_LUT_MAX_DAY_NUM), ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; };
} }
column_data.emplace_back(days_num); column_data.emplace_back(days_num);
@ -263,23 +271,47 @@ namespace DB
offsets_data.emplace_back(start + arrow_offsets.Value(i)); offsets_data.emplace_back(start + arrow_offsets.Value(i));
} }
} }
static ColumnPtr createAndFillColumnWithIndexesData(std::shared_ptr<arrow::ChunkedArray> & arrow_column)
{
switch (arrow_column->type()->id())
{
# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \
case ARROW_NUMERIC_TYPE: \
{ \
auto column = DataTypeNumber<CPP_NUMERIC_TYPE>().createColumn(); \
fillColumnWithNumericData<CPP_NUMERIC_TYPE>(arrow_column, *column); \
return column; \
}
FOR_ARROW_INDEXES_TYPES(DISPATCH)
# undef DISPATCH
default:
throw Exception(fmt::format("Unsupported type for indexes in LowCardinality: {}.", arrow_column->type()->name()), ErrorCodes::BAD_ARGUMENTS);
}
}
static void readColumnFromArrowColumn(std::shared_ptr<arrow::ChunkedArray> & arrow_column, IColumn & internal_column, const std::string & column_name, const std::string format_name, bool is_nullable) static void readColumnFromArrowColumn(
std::shared_ptr<arrow::ChunkedArray> & arrow_column,
IColumn & internal_column,
const std::string & column_name,
const std::string & format_name,
bool is_nullable,
std::unordered_map<String, ColumnPtr> dictionary_values)
{ {
if (internal_column.isNullable()) if (internal_column.isNullable())
{ {
ColumnNullable & column_nullable = typeid_cast<ColumnNullable &>(internal_column); ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(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()); fillByteMapFromArrowColumn(arrow_column, column_nullable.getNullMapColumn());
return; return;
} }
// TODO: check if a column is const? /// TODO: check if a column is const?
if (!is_nullable && !checkColumn<ColumnArray>(internal_column) && arrow_column->null_count()) 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 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 ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN
}; };
} }
@ -304,13 +336,11 @@ namespace DB
fillColumnWithTimestampData(arrow_column, internal_column); fillColumnWithTimestampData(arrow_column, internal_column);
break; break;
case arrow::Type::DECIMAL: case arrow::Type::DECIMAL:
//fillColumnWithNumericData<Decimal128, ColumnDecimal<Decimal128>>(arrow_column, read_column); // Have problems with trash values under NULL, but faster
fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/); fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/);
break; break;
case arrow::Type::MAP: [[fallthrough]];
case arrow::Type::LIST: case arrow::Type::LIST:
{ {
const auto * list_type = static_cast<arrow::ListType *>(arrow_column->type().get());
auto list_nested_type = list_type->value_type();
arrow::ArrayVector array_vector; arrow::ArrayVector array_vector;
array_vector.reserve(arrow_column->num_chunks()); array_vector.reserve(arrow_column->num_chunks());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
@ -321,11 +351,70 @@ namespace DB
} }
auto arrow_nested_column = std::make_shared<arrow::ChunkedArray>(array_vector); auto arrow_nested_column = std::make_shared<arrow::ChunkedArray>(array_vector);
ColumnArray & column_array = typeid_cast<ColumnArray &>(internal_column); ColumnArray & column_array = arrow_column->type()->id() == arrow::Type::MAP
readColumnFromArrowColumn(arrow_nested_column, column_array.getData(), column_name, format_name, false); ? assert_cast<ColumnMap &>(internal_column).getNestedColumn()
: assert_cast<ColumnArray &>(internal_column);
readColumnFromArrowColumn(arrow_nested_column, column_array.getData(), column_name, format_name, false, dictionary_values);
fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn()); fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn());
break; break;
} }
case arrow::Type::STRUCT:
{
ColumnTuple & column_tuple = assert_cast<ColumnTuple &>(internal_column);
int fields_count = column_tuple.tupleSize();
std::vector<arrow::ArrayVector> nested_arrow_columns(fields_count);
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
arrow::StructArray & struct_chunk = static_cast<arrow::StructArray &>(*(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<arrow::ChunkedArray>(nested_arrow_columns[i]);
readColumnFromArrowColumn(nested_arrow_column, column_tuple.getColumn(i), column_name, format_name, false, dictionary_values);
}
break;
}
case arrow::Type::DICTIONARY:
{
ColumnLowCardinality & column_lc = assert_cast<ColumnLowCardinality &>(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<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
arrow::DictionaryArray & dict_chunk = static_cast<arrow::DictionaryArray &>(*(arrow_column->chunk(chunk_i)));
dict_array.emplace_back(dict_chunk.dictionary());
}
auto arrow_dict_column = std::make_shared<arrow::ChunkedArray>(dict_array);
auto dict_column = IColumn::mutate(column_lc.getDictionaryPtr());
auto * uniq_column = static_cast<IColumnUnique *>(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<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
arrow::DictionaryArray & dict_chunk = static_cast<arrow::DictionaryArray &>(*(arrow_column->chunk(chunk_i)));
indexes_array.emplace_back(dict_chunk.indices());
}
auto arrow_indexes_column = std::make_shared<arrow::ChunkedArray>(indexes_array);
auto indexes_column = createAndFillColumnWithIndexesData(arrow_indexes_column);
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) \ # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \
case ARROW_NUMERIC_TYPE: \ case ARROW_NUMERIC_TYPE: \
fillColumnWithNumericData<CPP_NUMERIC_TYPE>(arrow_column, internal_column); \ fillColumnWithNumericData<CPP_NUMERIC_TYPE>(arrow_column, internal_column); \
@ -339,8 +428,7 @@ namespace DB
default: default:
throw Exception throw Exception
{ {
"Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" fmt::format(R"(Unsupported {} type "{}" of an input column "{}".)", format_name, arrow_column->type()->name(), column_name),
+ column_name + "\"",
ErrorCodes::UNKNOWN_TYPE ErrorCodes::UNKNOWN_TYPE
}; };
} }
@ -350,7 +438,7 @@ namespace DB
{ {
if (column_type->isNullable()) if (column_type->isNullable())
{ {
DataTypePtr nested_type = typeid_cast<const DataTypeNullable *>(column_type.get())->getNestedType(); DataTypePtr nested_type = assert_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
return makeNullable(getInternalType(arrow_type, nested_type, column_name, format_name)); return makeNullable(getInternalType(arrow_type, nested_type, column_name, format_name));
} }
@ -367,11 +455,61 @@ namespace DB
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(column_type.get()); const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(column_type.get());
if (!array_type) 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<DataTypeArray>(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name)); return std::make_shared<DataTypeArray>(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::StructType *>(arrow_type.get());
const DataTypeTuple * tuple_type = typeid_cast<const DataTypeTuple *>(column_type.get());
if (!tuple_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();
/// 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
{
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
};
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<DataTypeTuple>(std::move(nested_types));
}
if (arrow_type->id() == arrow::Type::DICTIONARY)
{
const auto * arrow_dict_type = static_cast<arrow::DictionaryType *>(arrow_type.get());
const auto * lc_type = typeid_cast<const DataTypeLowCardinality *>(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<DataTypeLowCardinality>(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::MapType *>(arrow_type.get());
const auto * map_type = typeid_cast<const DataTypeMap *>(column_type.get());
if (!map_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<DataTypeMap>(
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(), 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(); }); [=](auto && elem) { return elem.first == arrow_type->id(); });
internal_type_it != arrow_type_to_internal_type.end()) internal_type_it != arrow_type_to_internal_type.end())
@ -380,13 +518,24 @@ namespace DB
} }
throw Exception 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(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 ErrorCodes::CANNOT_CONVERT_TYPE
}; };
} }
void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr<arrow::Table> & table, ArrowColumnToCHColumn::ArrowColumnToCHColumn(const Block & header_, std::shared_ptr<arrow::Schema> schema_, const std::string & format_name_) : header(header_), format_name(format_name_)
const Block & header, std::string 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<arrow::Table> & table)
{ {
Columns columns_list; Columns columns_list;
UInt64 num_rows = 0; UInt64 num_rows = 0;
@ -404,20 +553,18 @@ namespace DB
for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i)
{ {
ColumnWithTypeAndName header_column = header.getByPosition(column_i); const ColumnWithTypeAndName & header_column = header.getByPosition(column_i);
const auto column_type = recursiveRemoveLowCardinality(header_column.type);
if (name_to_column_ptr.find(header_column.name) == name_to_column_ptr.end()) 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? // 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}; ErrorCodes::THERE_IS_NO_COLUMN};
std::shared_ptr<arrow::ChunkedArray> arrow_column = name_to_column_ptr[header_column.name]; std::shared_ptr<arrow::ChunkedArray> 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(); 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; ColumnWithTypeAndName column;
column.name = header_column.name; column.name = header_column.name;

View File

@ -19,11 +19,15 @@
namespace DB namespace DB
{ {
class ArrowColumnToCHColumn class ArrowColumnToCHColumn
{ {
private: public:
ArrowColumnToCHColumn(const Block & header_, std::shared_ptr<arrow::Schema> schema_, const std::string & format_name_);
# define FOR_ARROW_NUMERIC_TYPES(M) \ void arrowTableToCHChunk(Chunk & res, std::shared_ptr<arrow::Table> & table);
private:
#define FOR_ARROW_NUMERIC_TYPES(M) \
M(arrow::Type::UINT8, DB::UInt8) \ M(arrow::Type::UINT8, DB::UInt8) \
M(arrow::Type::INT8, DB::Int8) \ M(arrow::Type::INT8, DB::Int8) \
M(arrow::Type::UINT16, DB::UInt16) \ M(arrow::Type::UINT16, DB::UInt16) \
@ -36,11 +40,24 @@ namespace DB
M(arrow::Type::FLOAT, DB::Float32) \ M(arrow::Type::FLOAT, DB::Float32) \
M(arrow::Type::DOUBLE, DB::Float64) M(arrow::Type::DOUBLE, DB::Float64)
#define FOR_ARROW_INDEXES_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)
public:
static void arrowTableToCHChunk(Chunk & res, std::shared_ptr<arrow::Table> & table, const Block & header;
const Block & header, std::string format_name); std::unordered_map<std::string, DataTypePtr> 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<std::string, ColumnPtr> dictionary_values;
};
} }
#endif #endif

View File

@ -6,17 +6,22 @@
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h> #include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnMap.h>
#include <Core/callOnTypeIndex.h> #include <Core/callOnTypeIndex.h>
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesDecimal.h> #include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
#include <Processors/Formats/IOutputFormat.h> #include <Processors/Formats/IOutputFormat.h>
#include <arrow/api.h> #include <arrow/api.h>
#include <arrow/builder.h> #include <arrow/builder.h>
#include <arrow/type.h> #include <arrow/type.h>
#include <arrow/util/decimal.h> #include <arrow/util/decimal.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace DB namespace DB
@ -25,6 +30,7 @@ namespace DB
{ {
extern const int UNKNOWN_EXCEPTION; extern const int UNKNOWN_EXCEPTION;
extern const int UNKNOWN_TYPE; extern const int UNKNOWN_TYPE;
extern const int LOGICAL_ERROR;
} }
static const std::initializer_list<std::pair<String, std::shared_ptr<arrow::DataType>>> internal_type_to_arrow_type = static const std::initializer_list<std::pair<String, std::shared_ptr<arrow::DataType>>> internal_type_to_arrow_type =
@ -46,16 +52,15 @@ namespace DB
//{"DateTime", arrow::date64()}, // BUG! saves as date32 //{"DateTime", arrow::date64()}, // BUG! saves as date32
{"DateTime", arrow::uint32()}, {"DateTime", arrow::uint32()},
// TODO: ClickHouse can actually store non-utf8 strings! {"String", arrow::binary()},
{"String", arrow::utf8()}, {"FixedString", arrow::binary()},
{"FixedString", arrow::utf8()},
}; };
static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name)
{ {
if (!status.ok()) 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 <typename NumericType, typename ArrowBuilderType> template <typename NumericType, typename ArrowBuilderType>
@ -101,8 +106,10 @@ namespace DB
arrow::ArrayBuilder * array_builder, arrow::ArrayBuilder * array_builder,
String format_name, String format_name,
size_t start, size_t start,
size_t end); size_t end,
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values);
template <typename Builder>
static void fillArrowArrayWithArrayColumnData( static void fillArrowArrayWithArrayColumnData(
const String & column_name, const String & column_name,
ColumnPtr & column, ColumnPtr & column,
@ -111,26 +118,164 @@ namespace DB
arrow::ArrayBuilder * array_builder, arrow::ArrayBuilder * array_builder,
String format_name, String format_name,
size_t start, size_t start,
size_t end) size_t end,
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
{ {
const auto * column_array = static_cast<const ColumnArray *>(column.get()); const auto * column_array = assert_cast<const ColumnArray *>(column.get());
ColumnPtr nested_column = column_array->getDataPtr(); ColumnPtr nested_column = column_array->getDataPtr();
DataTypePtr nested_type = typeid_cast<const DataTypeArray *>(column_type.get())->getNestedType(); DataTypePtr nested_type = assert_cast<const DataTypeArray *>(column_type.get())->getNestedType();
const auto & offsets = column_array->getOffsets(); const auto & offsets = column_array->getOffsets();
arrow::ListBuilder & builder = assert_cast<arrow::ListBuilder &>(*array_builder); Builder & builder = assert_cast<Builder &>(*array_builder);
arrow::ArrayBuilder * value_builder = builder.value_builder(); arrow::ArrayBuilder * value_builder = builder.value_builder();
arrow::Status components_status; arrow::Status components_status;
for (size_t array_idx = start; array_idx < end; ++array_idx) for (size_t array_idx = start; array_idx < end; ++array_idx)
{ {
/// Start new array /// Start new array.
components_status = builder.Append(); components_status = builder.Append();
checkStatus(components_status, nested_column->getName(), format_name); 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);
} }
} }
static void fillArrowArrayWithTupleColumnData(
const String & column_name,
ColumnPtr & column,
const std::shared_ptr<const IDataType> & column_type,
const PaddedPODArray<UInt8> * null_bytemap,
arrow::ArrayBuilder * array_builder,
String format_name,
size_t start,
size_t end,
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
{
const auto * column_tuple = assert_cast<const ColumnTuple *>(column.get());
const auto & nested_types = assert_cast<const DataTypeTuple *>(column_type.get())->getElements();
arrow::StructBuilder & builder = assert_cast<arrow::StructBuilder &>(*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, dictionary_values);
}
for (size_t i = start; i != end; ++i)
{
auto status = builder.Append();
checkStatus(status, column->getName(), format_name);
}
}
template<typename T>
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end)
{
const PaddedPODArray<T> & data = assert_cast<const ColumnVector<T> *>(column.get())->getData();
PaddedPODArray<Int64> 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<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end)
{
switch (column->getDataType())
{
case TypeIndex::UInt8:
return extractIndexesImpl<UInt8>(column, start, end);
case TypeIndex::UInt16:
return extractIndexesImpl<UInt16>(column, start, end);
case TypeIndex::UInt32:
return extractIndexesImpl<UInt32>(column, start, end);
case TypeIndex::UInt64:
return extractIndexesImpl<UInt64>(column, start, end);
default:
throw Exception(fmt::format("Indexes column must be ColumnUInt, got {}.", column->getName()),
ErrorCodes::LOGICAL_ERROR);
}
}
template<typename ValueType>
static void fillArrowArrayWithLowCardinalityColumnDataImpl(
const String & column_name,
ColumnPtr & column,
const std::shared_ptr<const IDataType> & column_type,
const PaddedPODArray<UInt8> * null_bytemap,
arrow::ArrayBuilder * array_builder,
String format_name,
size_t start,
size_t end,
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
{
const auto * column_lc = assert_cast<const ColumnLowCardinality *>(column.get());
arrow::DictionaryBuilder<ValueType> * builder = assert_cast<arrow::DictionaryBuilder<ValueType> *>(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 = assert_cast<arrow::DictionaryType *>(builder->type().get())->value_type();
std::unique_ptr<arrow::ArrayBuilder> 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 = assert_cast<const DataTypeLowCardinality *>(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<uint8_t> 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<const IDataType> & column_type,
const PaddedPODArray<UInt8> * null_bytemap,
arrow::ArrayBuilder * array_builder,
String format_name,
size_t start,
size_t end,
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
{
auto value_type = assert_cast<arrow::DictionaryType *>(array_builder->type().get())->value_type();
#define DISPATCH(ARROW_TYPE_ID, ARROW_TYPE) \
if (arrow::Type::ARROW_TYPE_ID == value_type->id()) \
{ \
fillArrowArrayWithLowCardinalityColumnDataImpl<ARROW_TYPE>(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end, dictionary_values); \
return; \
}
FOR_ARROW_TYPES(DISPATCH)
#undef DISPATCH
}
template <typename ColumnType> template <typename ColumnType>
static void fillArrowArrayWithStringColumnData( static void fillArrowArrayWithStringColumnData(
ColumnPtr write_column, ColumnPtr write_column,
@ -141,7 +286,7 @@ namespace DB
size_t end) size_t end)
{ {
const auto & internal_column = assert_cast<const ColumnType &>(*write_column); const auto & internal_column = assert_cast<const ColumnType &>(*write_column);
arrow::StringBuilder & builder = assert_cast<arrow::StringBuilder &>(*array_builder); arrow::BinaryBuilder & builder = assert_cast<arrow::BinaryBuilder &>(*array_builder);
arrow::Status status; arrow::Status status;
for (size_t string_i = start; string_i < end; ++string_i) for (size_t string_i = start; string_i < end; ++string_i)
@ -155,7 +300,6 @@ namespace DB
StringRef string_ref = internal_column.getDataAt(string_i); StringRef string_ref = internal_column.getDataAt(string_i);
status = builder.Append(string_ref.data, string_ref.size); status = builder.Append(string_ref.data, string_ref.size);
} }
checkStatus(status, write_column->getName(), format_name); checkStatus(status, write_column->getName(), format_name);
} }
} }
@ -218,18 +362,19 @@ namespace DB
arrow::ArrayBuilder * array_builder, arrow::ArrayBuilder * array_builder,
String format_name, String format_name,
size_t start, size_t start,
size_t end) size_t end,
std::unordered_map<String, std::shared_ptr<arrow::Array>> & dictionary_values)
{ {
const String column_type_name = column_type->getFamilyName(); const String column_type_name = column_type->getFamilyName();
if ("Nullable" == column_type_name) if ("Nullable" == column_type_name)
{ {
const ColumnNullable * column_nullable = checkAndGetColumn<ColumnNullable>(column.get()); const ColumnNullable * column_nullable = assert_cast<const ColumnNullable *>(column.get());
ColumnPtr nested_column = column_nullable->getNestedColumnPtr(); ColumnPtr nested_column = column_nullable->getNestedColumnPtr();
DataTypePtr nested_type = typeid_cast<const DataTypeNullable *>(column_type.get())->getNestedType(); DataTypePtr nested_type = assert_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
const ColumnPtr & null_column = column_nullable->getNullMapColumnPtr(); const ColumnPtr & null_column = column_nullable->getNullMapColumnPtr();
const PaddedPODArray<UInt8> & bytemap = assert_cast<const ColumnVector<UInt8> &>(*null_column).getData(); const PaddedPODArray<UInt8> & bytemap = assert_cast<const ColumnVector<UInt8> &>(*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) else if ("String" == column_type_name)
{ {
@ -249,7 +394,21 @@ namespace DB
} }
else if ("Array" == column_type_name) else if ("Array" == column_type_name)
{ {
fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end); fillArrowArrayWithArrayColumnData<arrow::ListBuilder>(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, 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 ("Map" == column_type_name)
{
ColumnPtr column_array = assert_cast<const ColumnMap *>(column.get())->getNestedColumnPtr();
DataTypePtr array_type = assert_cast<const DataTypeMap *>(column_type.get())->getNestedType();
fillArrowArrayWithArrayColumnData<arrow::MapBuilder>(column_name, column_array, array_type, null_bytemap, array_builder, format_name, start, end, dictionary_values);
} }
else if (isDecimal(column_type)) else if (isDecimal(column_type))
{ {
@ -280,7 +439,7 @@ namespace DB
{ {
throw Exception 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(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 ErrorCodes::UNKNOWN_TYPE
}; };
} }
@ -295,7 +454,7 @@ namespace DB
size_t start, size_t start,
size_t end) size_t end)
{ {
const auto & column = static_cast<const typename DataType::ColumnType &>(*write_column); const auto & column = assert_cast<const typename DataType::ColumnType &>(*write_column);
arrow::DecimalBuilder & builder = assert_cast<arrow::DecimalBuilder &>(*array_builder); arrow::DecimalBuilder & builder = assert_cast<arrow::DecimalBuilder &>(*array_builder);
arrow::Status status; arrow::Status status;
@ -312,12 +471,33 @@ namespace DB
checkStatus(status, write_column->getName(), format_name); checkStatus(status, write_column->getName(), format_name);
} }
static std::shared_ptr<arrow::DataType> getArrowType(DataTypePtr column_type, const std::string & column_name, const std::string & format_name, bool * is_column_nullable) static std::shared_ptr<arrow::DataType> 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(fmt::format("Indexes column for getUniqueIndex must be ColumnUInt, got {}.", indexes_column->getName()),
ErrorCodes::LOGICAL_ERROR);
}
}
static std::shared_ptr<arrow::DataType> getArrowType(DataTypePtr column_type, ColumnPtr column, const std::string & column_name, const std::string & format_name, bool * is_column_nullable)
{ {
if (column_type->isNullable()) if (column_type->isNullable())
{ {
DataTypePtr nested_type = typeid_cast<const DataTypeNullable *>(column_type.get())->getNestedType(); DataTypePtr nested_type = assert_cast<const DataTypeNullable *>(column_type.get())->getNestedType();
auto arrow_type = getArrowType(nested_type, column_name, format_name, is_column_nullable); ColumnPtr nested_column = assert_cast<const ColumnNullable *>(column.get())->getNestedColumnPtr();
auto arrow_type = getArrowType(nested_type, nested_column, column_name, format_name, is_column_nullable);
*is_column_nullable = true; *is_column_nullable = true;
return arrow_type; return arrow_type;
} }
@ -334,7 +514,7 @@ namespace DB
|| std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> || std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>>
|| std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>) || std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
{ {
const auto & decimal_type = static_cast<const ToDataType *>(column_type.get()); const auto & decimal_type = assert_cast<const ToDataType *>(column_type.get());
arrow_type = arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale()); arrow_type = arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale());
} }
@ -346,11 +526,50 @@ namespace DB
if (isArray(column_type)) if (isArray(column_type))
{ {
auto nested_type = typeid_cast<const DataTypeArray *>(column_type.get())->getNestedType(); auto nested_type = assert_cast<const DataTypeArray *>(column_type.get())->getNestedType();
auto nested_arrow_type = getArrowType(nested_type, column_name, format_name, is_column_nullable); auto nested_column = assert_cast<const ColumnArray *>(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); return arrow::list(nested_arrow_type);
} }
if (isTuple(column_type))
{
const auto & nested_types = assert_cast<const DataTypeTuple *>(column_type.get())->getElements();
const auto * tuple_column = assert_cast<const ColumnTuple *>(column.get());
std::vector<std::shared_ptr<arrow::Field>> 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], tuple_column->getColumnPtr(i), name, format_name, is_column_nullable);
nested_fields.push_back(std::make_shared<arrow::Field>(name, nested_arrow_type, *is_column_nullable));
}
return arrow::struct_(std::move(nested_fields));
}
if (column_type->lowCardinality())
{
auto nested_type = assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
const auto * lc_column = assert_cast<const ColumnLowCardinality *>(column.get());
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));
}
if (isMap(column_type))
{
const auto * map_type = assert_cast<const DataTypeMap *>(column_type.get());
const auto & key_type = map_type->getKeyType();
const auto & val_type = map_type->getValueType();
const auto & columns = assert_cast<const ColumnMap *>(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(); const std::string type_name = column_type->getFamilyName();
if (const auto * arrow_type_it = std::find_if( if (const auto * arrow_type_it = std::find_if(
internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.begin(),
@ -361,49 +580,59 @@ namespace DB
return arrow_type_it->second; return arrow_type_it->second;
} }
throw Exception{"The type \"" + column_name + "\" of a column \"" + column_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),
" is not supported for conversion into a " + format_name + " data format",
ErrorCodes::UNKNOWN_TYPE}; 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<arrow::Field>(column.name, arrow_type, is_column_nullable));
header_columns.emplace_back(std::move(column));
}
}
void CHColumnToArrowColumn::chChunkToArrowTable( void CHColumnToArrowColumn::chChunkToArrowTable(
std::shared_ptr<arrow::Table> & res, std::shared_ptr<arrow::Table> & res,
const Block & header,
const Chunk & chunk, const Chunk & chunk,
size_t columns_num, size_t columns_num)
String format_name)
{ {
/// For arrow::Schema and arrow::Table creation /// For arrow::Schema and arrow::Table creation
std::vector<std::shared_ptr<arrow::Field>> arrow_fields;
std::vector<std::shared_ptr<arrow::Array>> arrow_arrays; std::vector<std::shared_ptr<arrow::Array>> arrow_arrays;
arrow_fields.reserve(columns_num);
arrow_arrays.reserve(columns_num); arrow_arrays.reserve(columns_num);
for (size_t column_i = 0; column_i < columns_num; ++column_i) for (size_t column_i = 0; column_i < columns_num; ++column_i)
{ {
// TODO: constructed every iteration const ColumnWithTypeAndName & header_column = header_columns[column_i];
ColumnWithTypeAndName column = header.safeGetByPosition(column_i); auto column = chunk.getColumns()[column_i];
column.column = recursiveRemoveLowCardinality(chunk.getColumns()[column_i]);
column.type = recursiveRemoveLowCardinality(column.type);
bool is_column_nullable = false; if (!low_cardinality_as_dictionary)
auto arrow_type = getArrowType(column.type, column.name, format_name, &is_column_nullable); column = recursiveRemoveLowCardinality(column);
arrow_fields.emplace_back(std::make_shared<arrow::Field>(column.name, arrow_type, is_column_nullable));
arrow::MemoryPool* pool = arrow::default_memory_pool(); arrow::MemoryPool* pool = arrow::default_memory_pool();
std::unique_ptr<arrow::ArrayBuilder> array_builder; std::unique_ptr<arrow::ArrayBuilder> array_builder;
arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &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()); fillArrowArray(header_column.name, column, header_column.type, nullptr, array_builder.get(), format_name, 0, column->size(), dictionary_values);
std::shared_ptr<arrow::Array> arrow_array; std::shared_ptr<arrow::Array> arrow_array;
status = array_builder->Finish(&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)); arrow_arrays.emplace_back(std::move(arrow_array));
} }
std::shared_ptr<arrow::Schema> arrow_schema = std::make_shared<arrow::Schema>(std::move(arrow_fields)); std::shared_ptr<arrow::Schema> arrow_schema = std::make_shared<arrow::Schema>(arrow_fields);
res = arrow::Table::Make(arrow_schema, arrow_arrays); res = arrow::Table::Make(arrow_schema, arrow_arrays);
} }

View File

@ -12,6 +12,10 @@ namespace DB
class CHColumnToArrowColumn class CHColumnToArrowColumn
{ {
public:
CHColumnToArrowColumn(const Block & header, const std::string & format_name_, bool low_cardinality_as_dictionary_ = false);
void chChunkToArrowTable(std::shared_ptr<arrow::Table> & res, const Chunk & chunk, size_t columns_num);
private: private:
#define FOR_INTERNAL_NUMERIC_TYPES(M) \ #define FOR_INTERNAL_NUMERIC_TYPES(M) \
@ -26,10 +30,27 @@ private:
M(Float32, arrow::FloatBuilder) \ M(Float32, arrow::FloatBuilder) \
M(Float64, arrow::DoubleBuilder) 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)
public: ColumnsWithTypeAndName header_columns;
static void chChunkToArrowTable(std::shared_ptr<arrow::Table> & res, const Block & header, const Chunk & chunk, std::vector<std::shared_ptr<arrow::Field>> arrow_fields;
size_t columns_num, String format_name); 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<std::string, std::shared_ptr<arrow::Array>> dictionary_values;
}; };
} }
#endif #endif

View File

@ -33,7 +33,6 @@ ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInp
Chunk ORCBlockInputFormat::generate() Chunk ORCBlockInputFormat::generate()
{ {
Chunk res; Chunk res;
const Block & header = getPort().getHeader();
if (!file_reader) if (!file_reader)
prepareReader(); prepareReader();
@ -54,7 +53,7 @@ Chunk ORCBlockInputFormat::generate()
++stripe_current; ++stripe_current;
ArrowColumnToCHColumn::arrowTableToCHChunk(res, *table_result, header, "ORC"); arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result);
return res; return res;
} }
@ -67,11 +66,26 @@ void ORCBlockInputFormat::resetParser()
stripe_current = 0; stripe_current = 0;
} }
size_t countIndicesForType(std::shared_ptr<arrow::DataType> type) static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
{ {
if (type->id() == arrow::Type::LIST) if (type->id() == arrow::Type::LIST)
return countIndicesForType(static_cast<arrow::ListType *>(type.get())->value_type()) + 1; return countIndicesForType(static_cast<arrow::ListType *>(type.get())->value_type()) + 1;
if (type->id() == arrow::Type::STRUCT)
{
int indices = 1;
auto * struct_type = static_cast<arrow::StructType *>(type.get());
for (int i = 0; i != struct_type->num_fields(); ++i)
indices += countIndicesForType(struct_type->field(i)->type());
return indices;
}
if (type->id() == arrow::Type::MAP)
{
auto * map_type = static_cast<arrow::MapType *>(type.get());
return countIndicesForType(map_type->key_type()) + countIndicesForType(map_type->item_type());
}
return 1; return 1;
} }
@ -84,17 +98,22 @@ void ORCBlockInputFormat::prepareReader()
std::shared_ptr<arrow::Schema> schema; std::shared_ptr<arrow::Schema> schema;
THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema));
int index = 0; arrow_column_to_ch_column = std::make_unique<ArrowColumnToCHColumn>(getPort().getHeader(), schema, "ORC");
/// 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) for (int i = 0; i < schema->num_fields(); ++i)
{ {
/// 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())) 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) for (int j = 0; j != indexes_count; ++j)
include_indices.push_back(index++); include_indices.push_back(index + j);
} }
index += indexes_count;
} }
} }

View File

@ -8,6 +8,9 @@ namespace arrow::adapters::orc { class ORCFileReader; }
namespace DB namespace DB
{ {
class ArrowColumnToCHColumn;
class ORCBlockInputFormat : public IInputFormat class ORCBlockInputFormat : public IInputFormat
{ {
public: public:
@ -26,6 +29,8 @@ private:
std::unique_ptr<arrow::adapters::orc::ORCFileReader> file_reader; std::unique_ptr<arrow::adapters::orc::ORCFileReader> file_reader;
std::unique_ptr<ArrowColumnToCHColumn> arrow_column_to_ch_column;
int stripe_total = 0; int stripe_total = 0;
int stripe_current = 0; int stripe_current = 0;

View File

@ -10,12 +10,16 @@
#include <Columns/ColumnVector.h> #include <Columns/ColumnVector.h>
#include <Columns/ColumnArray.h> #include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h> #include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesDecimal.h> #include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
namespace DB namespace DB
{ {
@ -46,15 +50,9 @@ void ORCOutputStream::write(const void* buf, size_t length)
ORCBlockOutputFormat::ORCBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) 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()) : 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<orc::Type> ORCBlockOutputFormat::getORCType(const DataTypePtr & type) ORC_UNIQUE_PTR<orc::Type> ORCBlockOutputFormat::getORCType(const DataTypePtr & type, const std::string & column_name)
{ {
switch (type->getTypeId()) switch (type->getTypeId())
{ {
@ -102,28 +100,48 @@ ORC_UNIQUE_PTR<orc::Type> ORCBlockOutputFormat::getORCType(const DataTypePtr & t
} }
case TypeIndex::Nullable: case TypeIndex::Nullable:
{ {
return getORCType(removeNullable(type)); return getORCType(removeNullable(type), column_name);
} }
case TypeIndex::Array: case TypeIndex::Array:
{ {
const auto * array_type = typeid_cast<const DataTypeArray *>(type.get()); const auto * array_type = assert_cast<const DataTypeArray *>(type.get());
return orc::createListType(getORCType(array_type->getNestedType())); return orc::createListType(getORCType(array_type->getNestedType(), column_name));
} }
case TypeIndex::Decimal32: case TypeIndex::Decimal32:
{ {
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal32> *>(type.get()); const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal32> *>(type.get());
return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale()); return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale());
} }
case TypeIndex::Decimal64: case TypeIndex::Decimal64:
{ {
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal64> *>(type.get()); const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal64> *>(type.get());
return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale()); return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale());
} }
case TypeIndex::Decimal128: case TypeIndex::Decimal128:
{ {
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal128> *>(type.get()); const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal128> *>(type.get());
return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale()); return orc::createDecimalType(decimal_type->getPrecision(), decimal_type->getScale());
} }
case TypeIndex::Tuple:
{
const auto * tuple_type = assert_cast<const DataTypeTuple *>(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;
}
case TypeIndex::Map:
{
const auto * map_type = assert_cast<const DataTypeMap *>(type.get());
return orc::createMapType(
getORCType(map_type->getKeyType(), column_name),
getORCType(map_type->getValueType(), column_name)
);
}
default: default:
{ {
throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN); throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN);
@ -149,6 +167,8 @@ void ORCBlockOutputFormat::writeNumbers(
number_orc_column.notNull[i] = 0; number_orc_column.notNull[i] = 0;
continue; continue;
} }
number_orc_column.notNull[i] = 1;
number_orc_column.data[i] = convert(number_column.getElement(i)); number_orc_column.data[i] = convert(number_column.getElement(i));
} }
number_orc_column.numElements = number_column.size(); number_orc_column.numElements = number_column.size();
@ -164,7 +184,7 @@ void ORCBlockOutputFormat::writeDecimals(
{ {
DecimalVectorBatch & decimal_orc_column = dynamic_cast<DecimalVectorBatch &>(orc_column); DecimalVectorBatch & decimal_orc_column = dynamic_cast<DecimalVectorBatch &>(orc_column);
const auto & decimal_column = assert_cast<const ColumnDecimal<Decimal> &>(column); const auto & decimal_column = assert_cast<const ColumnDecimal<Decimal> &>(column);
const auto * decimal_type = typeid_cast<const DataTypeDecimal<Decimal> *>(type.get()); const auto * decimal_type = assert_cast<const DataTypeDecimal<Decimal> *>(type.get());
decimal_orc_column.precision = decimal_type->getPrecision(); decimal_orc_column.precision = decimal_type->getPrecision();
decimal_orc_column.scale = decimal_type->getScale(); decimal_orc_column.scale = decimal_type->getScale();
decimal_orc_column.resize(decimal_column.size()); decimal_orc_column.resize(decimal_column.size());
@ -175,6 +195,8 @@ void ORCBlockOutputFormat::writeDecimals(
decimal_orc_column.notNull[i] = 0; decimal_orc_column.notNull[i] = 0;
continue; continue;
} }
decimal_orc_column.notNull[i] = 1;
decimal_orc_column.values[i] = convert(decimal_column.getElement(i).value); decimal_orc_column.values[i] = convert(decimal_column.getElement(i).value);
} }
decimal_orc_column.numElements = decimal_column.size(); decimal_orc_column.numElements = decimal_column.size();
@ -197,6 +219,8 @@ void ORCBlockOutputFormat::writeStrings(
string_orc_column.notNull[i] = 0; string_orc_column.notNull[i] = 0;
continue; continue;
} }
string_orc_column.notNull[i] = 1;
const StringRef & string = string_column.getDataAt(i); const StringRef & string = string_column.getDataAt(i);
string_orc_column.data[i] = const_cast<char *>(string.data); string_orc_column.data[i] = const_cast<char *>(string.data);
string_orc_column.length[i] = string.size; string_orc_column.length[i] = string.size;
@ -223,6 +247,8 @@ void ORCBlockOutputFormat::writeDateTimes(
timestamp_orc_column.notNull[i] = 0; timestamp_orc_column.notNull[i] = 0;
continue; continue;
} }
timestamp_orc_column.notNull[i] = 1;
timestamp_orc_column.data[i] = get_seconds(timestamp_column.getElement(i)); timestamp_orc_column.data[i] = get_seconds(timestamp_column.getElement(i));
timestamp_orc_column.nanoseconds[i] = get_nanoseconds(timestamp_column.getElement(i)); timestamp_orc_column.nanoseconds[i] = get_nanoseconds(timestamp_column.getElement(i));
} }
@ -235,11 +261,10 @@ void ORCBlockOutputFormat::writeColumn(
DataTypePtr & type, DataTypePtr & type,
const PaddedPODArray<UInt8> * null_bytemap) const PaddedPODArray<UInt8> * null_bytemap)
{ {
if (null_bytemap)
{
orc_column.hasNulls = true;
orc_column.notNull.resize(column.size()); orc_column.notNull.resize(column.size());
} if (null_bytemap)
orc_column.hasNulls = true;
switch (type->getTypeId()) switch (type->getTypeId())
{ {
case TypeIndex::Int8: case TypeIndex::Int8:
@ -374,12 +399,52 @@ void ORCBlockOutputFormat::writeColumn(
for (size_t i = 0; i != list_column.size(); ++i) for (size_t i = 0; i != list_column.size(); ++i)
{ {
list_orc_column.offsets[i + 1] = offsets[i]; list_orc_column.offsets[i + 1] = offsets[i];
list_orc_column.notNull[i] = 1;
} }
orc::ColumnVectorBatch & nested_orc_column = *list_orc_column.elements; orc::ColumnVectorBatch & nested_orc_column = *list_orc_column.elements;
writeColumn(nested_orc_column, list_column.getData(), nested_type, null_bytemap); writeColumn(nested_orc_column, list_column.getData(), nested_type, null_bytemap);
list_orc_column.numElements = list_column.size(); list_orc_column.numElements = list_column.size();
break; break;
} }
case TypeIndex::Tuple:
{
orc::StructVectorBatch & struct_orc_column = dynamic_cast<orc::StructVectorBatch &>(orc_column);
const auto & tuple_column = assert_cast<const ColumnTuple &>(column);
auto nested_types = assert_cast<const DataTypeTuple *>(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::MapVectorBatch &>(orc_column);
const auto & list_column = assert_cast<const ColumnMap &>(column).getNestedColumn();
const auto & map_type = assert_cast<const DataTypeMap &>(*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<const ColumnTuple *>(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: default:
throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN); throw Exception("Type " + type->getName() + " is not supported for ORC output format", ErrorCodes::ILLEGAL_COLUMN);
} }
@ -409,6 +474,8 @@ size_t ORCBlockOutputFormat::getMaxColumnSize(Chunk & chunk)
void ORCBlockOutputFormat::consume(Chunk chunk) void ORCBlockOutputFormat::consume(Chunk chunk)
{ {
if (!writer)
prepareWriter();
size_t columns_num = chunk.getNumColumns(); size_t columns_num = chunk.getNumColumns();
size_t rows_num = chunk.getNumRows(); size_t rows_num = chunk.getNumRows();
/// getMaxColumnSize is needed to write arrays. /// getMaxColumnSize is needed to write arrays.
@ -425,9 +492,23 @@ void ORCBlockOutputFormat::consume(Chunk chunk)
void ORCBlockOutputFormat::finalize() void ORCBlockOutputFormat::finalize()
{ {
if (!writer)
prepareWriter();
writer->close(); 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) void registerOutputFormatProcessorORC(FormatFactory & factory)
{ {
factory.registerOutputFormatProcessor("ORC", []( factory.registerOutputFormatProcessor("ORC", [](

View File

@ -43,7 +43,7 @@ public:
void finalize() override; void finalize() override;
private: private:
ORC_UNIQUE_PTR<orc::Type> getORCType(const DataTypePtr & type); ORC_UNIQUE_PTR<orc::Type> getORCType(const DataTypePtr & type, const std::string & column_name);
/// ConvertFunc is needed for type UInt8, because firstly UInt8 (char8_t) must be /// ConvertFunc is needed for type UInt8, because firstly UInt8 (char8_t) must be
/// converted to unsigned char (bugprone-signed-char-misuse in clang). /// 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 getColumnSize(const IColumn & column, DataTypePtr & type);
size_t getMaxColumnSize(Chunk & chunk); size_t getMaxColumnSize(Chunk & chunk);
void prepareWriter();
const FormatSettings format_settings; const FormatSettings format_settings;
ORCOutputStream output_stream; ORCOutputStream output_stream;
DataTypes data_types; DataTypes data_types;

View File

@ -38,7 +38,6 @@ ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_
Chunk ParquetBlockInputFormat::generate() Chunk ParquetBlockInputFormat::generate()
{ {
Chunk res; Chunk res;
const Block & header = getPort().getHeader();
if (!file_reader) if (!file_reader)
prepareReader(); prepareReader();
@ -54,7 +53,7 @@ Chunk ParquetBlockInputFormat::generate()
++row_group_current; ++row_group_current;
ArrowColumnToCHColumn::arrowTableToCHChunk(res, table, header, "Parquet"); arrow_column_to_ch_column->arrowTableToCHChunk(res, table);
return res; return res;
} }
@ -67,6 +66,29 @@ void ParquetBlockInputFormat::resetParser()
row_group_current = 0; row_group_current = 0;
} }
static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
{
if (type->id() == arrow::Type::LIST)
return countIndicesForType(static_cast<arrow::ListType *>(type.get())->value_type());
if (type->id() == arrow::Type::STRUCT)
{
int indices = 0;
auto * struct_type = static_cast<arrow::StructType *>(type.get());
for (int i = 0; i != struct_type->num_fields(); ++i)
indices += countIndicesForType(struct_type->field(i)->type());
return indices;
}
if (type->id() == arrow::Type::MAP)
{
auto * map_type = static_cast<arrow::MapType *>(type.get());
return countIndicesForType(map_type->key_type()) + countIndicesForType(map_type->item_type());
}
return 1;
}
void ParquetBlockInputFormat::prepareReader() void ParquetBlockInputFormat::prepareReader()
{ {
THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(in), arrow::default_memory_pool(), &file_reader)); THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(asArrowFile(in), arrow::default_memory_pool(), &file_reader));
@ -76,12 +98,21 @@ void ParquetBlockInputFormat::prepareReader()
std::shared_ptr<arrow::Schema> schema; std::shared_ptr<arrow::Schema> schema;
THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema)); THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema));
arrow_column_to_ch_column = std::make_unique<ArrowColumnToCHColumn>(getPort().getHeader(), schema, "Parquet");
int index = 0;
for (int i = 0; i < schema->num_fields(); ++i) 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())) if (getPort().getHeader().has(schema->field(i)->name()))
{ {
column_indices.push_back(i); for (int j = 0; j != indexes_count; ++j)
column_indices.push_back(index + j);
} }
index += indexes_count;
} }
} }

View File

@ -12,6 +12,8 @@ namespace arrow { class Buffer; }
namespace DB namespace DB
{ {
class ArrowColumnToCHColumn;
class ParquetBlockInputFormat : public IInputFormat class ParquetBlockInputFormat : public IInputFormat
{ {
public: public:
@ -32,6 +34,7 @@ private:
int row_group_total = 0; int row_group_total = 0;
// indices of columns to read from Parquet file // indices of columns to read from Parquet file
std::vector<int> column_indices; std::vector<int> column_indices;
std::unique_ptr<ArrowColumnToCHColumn> arrow_column_to_ch_column;
int row_group_current = 0; int row_group_current = 0;
}; };

View File

@ -13,7 +13,6 @@
#include <arrow/api.h> #include <arrow/api.h>
#include <arrow/util/memory.h> #include <arrow/util/memory.h>
#include <parquet/arrow/writer.h> #include <parquet/arrow/writer.h>
#include <parquet/deprecated_io.h>
#include "ArrowBufferedStreams.h" #include "ArrowBufferedStreams.h"
#include "CHColumnToArrowColumn.h" #include "CHColumnToArrowColumn.h"
@ -32,11 +31,16 @@ ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Blo
void ParquetBlockOutputFormat::consume(Chunk chunk) void ParquetBlockOutputFormat::consume(Chunk chunk)
{ {
const Block & header = getPort(PortKind::Main).getHeader();
const size_t columns_num = chunk.getNumColumns(); const size_t columns_num = chunk.getNumColumns();
std::shared_ptr<arrow::Table> arrow_table; std::shared_ptr<arrow::Table> arrow_table;
CHColumnToArrowColumn::chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet"); if (!ch_column_to_arrow_column)
{
const Block & header = getPort(PortKind::Main).getHeader();
ch_column_to_arrow_column = std::make_unique<CHColumnToArrowColumn>(header, "Parquet");
}
ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, chunk, columns_num);
if (!file_writer) if (!file_writer)
{ {

View File

@ -21,6 +21,9 @@ namespace arrow
namespace DB namespace DB
{ {
class CHColumnToArrowColumn;
class ParquetBlockOutputFormat : public IOutputFormat class ParquetBlockOutputFormat : public IOutputFormat
{ {
public: public:
@ -36,6 +39,7 @@ private:
const FormatSettings format_settings; const FormatSettings format_settings;
std::unique_ptr<parquet::arrow::FileWriter> file_writer; std::unique_ptr<parquet::arrow::FileWriter> file_writer;
std::unique_ptr<CHColumnToArrowColumn> ch_column_to_arrow_column;
}; };
} }

View File

@ -190,9 +190,10 @@ protected:
/// Initially reserved virtual column name may be shadowed by real column. /// Initially reserved virtual column name may be shadowed by real column.
bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const; bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const;
private: private:
StorageID storage_id; StorageID storage_id;
mutable std::mutex id_mutex; mutable std::mutex id_mutex;
/// Multiversion storage metadata. Allows to read/write storage metadata /// Multiversion storage metadata. Allows to read/write storage metadata

View File

@ -223,6 +223,12 @@ public:
DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor 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 TTLInfo = MergeTreeDataPartTTLInfo;
using TTLInfos = MergeTreeDataPartTTLInfos; using TTLInfos = MergeTreeDataPartTTLInfos;

View File

@ -2156,8 +2156,7 @@ bool MergeTreeData::renameTempPartAndReplace(
LOG_TRACE(log, "Renaming temporary part {} to {}.", part->relative_path, part_name); LOG_TRACE(log, "Renaming temporary part {} to {}.", part->relative_path, part_name);
auto it_duplicate = data_parts_by_info.find(part_info); if (auto it_duplicate = data_parts_by_info.find(part_info); it_duplicate != data_parts_by_info.end())
if (it_duplicate != data_parts_by_info.end())
{ {
String message = "Part " + (*it_duplicate)->getNameWithState() + " already exists"; String message = "Part " + (*it_duplicate)->getNameWithState() + " already exists";

View File

@ -402,6 +402,7 @@ public:
/// Returns a copy of the list so that the caller shouldn't worry about locks. /// Returns a copy of the list so that the caller shouldn't worry about locks.
DataParts getDataParts(const DataPartStates & affordable_states) const; DataParts getDataParts(const DataPartStates & affordable_states) const;
/// Returns sorted list of the parts with specified states /// Returns sorted list of the parts with specified states
/// out_states will contain snapshot of each part state /// out_states will contain snapshot of each part state
DataPartsVector getDataPartsVector( DataPartsVector getDataPartsVector(

View File

@ -262,8 +262,8 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(
{ {
log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART; log_entry.type = StorageReplicatedMergeTree::LogEntry::ATTACH_PART;
/// We don't need to involve ZooKeeper to obtain the checksums as by the time we get /// We don't need to involve ZooKeeper to obtain checksums as by the time we get
/// the MutableDataPartPtr here, we already have the data thus being able to /// MutableDataPartPtr here, we already have the data thus being able to
/// calculate the checksums. /// calculate the checksums.
log_entry.part_checksum = part->checksums.getTotalChecksumHex(); 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. 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; bool renamed = false;
try try
{ {
renamed = storage.renameTempPartAndAdd(part, nullptr, &transaction); renamed = storage.renameTempPartAndAdd(part, nullptr, &transaction);
@ -394,6 +395,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(
&& e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
throw; throw;
} }
if (!renamed) if (!renamed)
{ {
if (is_already_existing_part) if (is_already_existing_part)

View File

@ -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) 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: * The argument for the engine should be:
* - (for Replicated) The path to the table in ZooKeeper * - (for Replicated) The path to the table in ZooKeeper
* - (for Replicated) Replica name in ZooKeeper * - (for Replicated) Replica name in ZooKeeper

View File

@ -288,6 +288,7 @@ void replaceConstantExpressions(
/// is one of the following: /// is one of the following:
/// - QueryProcessingStage::Complete /// - QueryProcessingStage::Complete
/// - QueryProcessingStage::WithMergeableStateAfterAggregation /// - QueryProcessingStage::WithMergeableStateAfterAggregation
/// - QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit
/// - none (in this case regular WithMergeableState should be used) /// - none (in this case regular WithMergeableState should be used)
std::optional<QueryProcessingStage::Enum> getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, bool extremes, const Block & sharding_key_block) std::optional<QueryProcessingStage::Enum> getOptimizedQueryProcessingStage(const SelectQueryInfo & query_info, bool extremes, const Block & sharding_key_block)
{ {
@ -349,13 +350,13 @@ std::optional<QueryProcessingStage::Enum> getOptimizedQueryProcessingStage(const
// ORDER BY // ORDER BY
const ASTPtr order_by = select.orderBy(); const ASTPtr order_by = select.orderBy();
if (order_by) if (order_by)
return QueryProcessingStage::WithMergeableStateAfterAggregation; return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
// LIMIT BY // LIMIT BY
// LIMIT // LIMIT
// OFFSET // OFFSET
if (select.limitBy() || select.limitLength() || select.limitOffset()) 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. // Only simple SELECT FROM GROUP BY sharding_key can use Complete state.
return QueryProcessingStage::Complete; return QueryProcessingStage::Complete;
@ -514,10 +515,22 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
if (settings.distributed_group_by_no_merge) if (settings.distributed_group_by_no_merge)
{ {
if (settings.distributed_group_by_no_merge == DISTRIBUTED_GROUP_BY_NO_MERGE_AFTER_AGGREGATION) 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 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; return QueryProcessingStage::Complete;
} }
}
if (settings.distributed_push_down_limit)
return QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit;
/// Nested distributed query cannot return Complete stage, /// Nested distributed query cannot return Complete stage,
/// since the parent query need to aggregate the results after. /// since the parent query need to aggregate the results after.

View File

@ -63,7 +63,7 @@ namespace
/* Recursive directory listing with matched paths as a result. /* Recursive directory listing with matched paths as a result.
* Have the same method in StorageHDFS. * Have the same method in StorageHDFS.
*/ */
std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match) std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match, size_t & total_bytes_to_read)
{ {
const size_t first_glob = for_match.find_first_of("*?{"); const size_t first_glob = for_match.find_first_of("*?{");
@ -91,6 +91,7 @@ std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_fo
{ {
if (re2::RE2::FullMatch(file_name, matcher)) if (re2::RE2::FullMatch(file_name, matcher))
{ {
total_bytes_to_read += fs::file_size(it->path());
result.push_back(it->path().string()); result.push_back(it->path().string());
} }
} }
@ -99,7 +100,7 @@ std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_fo
if (re2::RE2::FullMatch(file_name, matcher)) if (re2::RE2::FullMatch(file_name, matcher))
{ {
/// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check.
Strings result_part = listFilesWithRegexpMatching(fs::path(full_path) / "", suffix_with_globs.substr(next_slash)); Strings result_part = listFilesWithRegexpMatching(fs::path(full_path) / "", suffix_with_globs.substr(next_slash), total_bytes_to_read);
std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
} }
} }
@ -127,7 +128,7 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di
} }
} }
Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context) Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read)
{ {
fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path); fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path);
fs::path fs_table_path(table_path); fs::path fs_table_path(table_path);
@ -137,9 +138,14 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user
Strings paths; Strings paths;
const String path = fs::weakly_canonical(fs_table_path); const String path = fs::weakly_canonical(fs_table_path);
if (path.find_first_of("*?{") == std::string::npos) if (path.find_first_of("*?{") == std::string::npos)
{
std::error_code error;
if (fs::exists(path))
total_bytes_to_read += fs::file_size(path, error);
paths.push_back(path); paths.push_back(path);
}
else else
paths = listFilesWithRegexpMatching("/", path); paths = listFilesWithRegexpMatching("/", path, total_bytes_to_read);
for (const auto & cur_path : paths) for (const auto & cur_path : paths)
checkCreationIsAllowed(context, user_files_absolute_path, cur_path); checkCreationIsAllowed(context, user_files_absolute_path, cur_path);
@ -173,7 +179,7 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us
: StorageFile(args) : StorageFile(args)
{ {
is_db_table = false; is_db_table = false;
paths = getPathsList(table_path_, user_files_path, args.getContext()); paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read);
if (args.format_name == "Distributed") if (args.format_name == "Distributed")
{ {
@ -361,6 +367,13 @@ public:
method = chooseCompressionMethod(current_path, storage->compression_method); method = chooseCompressionMethod(current_path, storage->compression_method);
} }
/// For clickhouse-local add progress callback to display progress bar.
if (context->getApplicationType() == Context::ApplicationType::LOCAL)
{
auto & in = static_cast<ReadBufferFromFileDescriptor &>(*nested_buffer);
in.setProgressCallback(context);
}
read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method); read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method);
auto get_block_for_format = [&]() -> Block auto get_block_for_format = [&]() -> Block
@ -418,6 +431,7 @@ public:
return {}; return {};
} }
private: private:
std::shared_ptr<StorageFile> storage; std::shared_ptr<StorageFile> storage;
StorageMetadataPtr metadata_snapshot; StorageMetadataPtr metadata_snapshot;
@ -480,6 +494,11 @@ Pipe StorageFile::read(
Pipes pipes; Pipes pipes;
pipes.reserve(num_streams); pipes.reserve(num_streams);
/// Set total number of bytes to process. For progress bar.
auto progress_callback = context->getFileProgressCallback();
if (context->getApplicationType() == Context::ApplicationType::LOCAL && progress_callback)
progress_callback(FileProgress(0, total_bytes_to_read));
for (size_t i = 0; i < num_streams; ++i) for (size_t i = 0; i < num_streams; ++i)
{ {
const auto get_columns_for_format = [&]() -> ColumnsDescription const auto get_columns_for_format = [&]() -> ColumnsDescription

View File

@ -58,7 +58,7 @@ public:
NamesAndTypesList getVirtuals() const override; NamesAndTypesList getVirtuals() const override;
static Strings getPathsList(const String & table_path, const String & user_files_path, ContextPtr context); static Strings getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read);
/// Check if the format is column-oriented. /// Check if the format is column-oriented.
/// Is is useful because column oriented formats could effectively skip unknown columns /// Is is useful because column oriented formats could effectively skip unknown columns
@ -103,6 +103,9 @@ private:
mutable std::shared_timed_mutex rwlock; mutable std::shared_timed_mutex rwlock;
Poco::Logger * log = &Poco::Logger::get("StorageFile"); Poco::Logger * log = &Poco::Logger::get("StorageFile");
/// Total number of bytes to read (sums for multiple files in case of globs). Needed for progress bar.
size_t total_bytes_to_read = 0;
}; };
} }

View File

@ -1,5 +1,6 @@
#include <Core/Defines.h> #include <Core/Defines.h>
#include "Common/hex.h"
#include <Common/Macros.h> #include <Common/Macros.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/ThreadPool.h> #include <Common/ThreadPool.h>
@ -63,10 +64,13 @@
#include <common/scope_guard.h> #include <common/scope_guard.h>
#include <common/scope_guard_safe.h> #include <common/scope_guard_safe.h>
#include <algorithm>
#include <ctime> #include <ctime>
#include <filesystem>
#include <iterator>
#include <numeric>
#include <thread> #include <thread>
#include <future> #include <future>
#include <filesystem>
#include <boost/algorithm/string/join.hpp> #include <boost/algorithm/string/join.hpp>
@ -135,6 +139,7 @@ namespace ErrorCodes
extern const int INTERSERVER_SCHEME_DOESNT_MATCH; extern const int INTERSERVER_SCHEME_DOESNT_MATCH;
extern const int DUPLICATE_DATA_PART; extern const int DUPLICATE_DATA_PART;
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
extern const int CONCURRENT_ACCESS_NOT_SUPPORTED;
} }
namespace ActionLocks 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_SLEEP_MS = 1 * 1000;
static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000; static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000;
std::atomic_uint StorageReplicatedMergeTree::total_fetches {0};
void StorageReplicatedMergeTree::setZooKeeper() void StorageReplicatedMergeTree::setZooKeeper()
{ {
/// Every ReplicatedMergeTree table is using only one ZooKeeper session. /// Every ReplicatedMergeTree table is using only one ZooKeeper session.
@ -376,7 +377,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
if (attach && !current_zookeeper->exists(zookeeper_path + "/metadata")) 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; is_readonly = true;
has_metadata_in_zookeeper = false; has_metadata_in_zookeeper = false;
return; return;
@ -384,10 +385,20 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
auto metadata_snapshot = getInMemoryMetadataPtr(); 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 (!attach)
{ {
if (!getDataParts().empty()) 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." " - probably it was unclean DROP table or manual intervention."
" You must either clear directory by hand or use ATTACH TABLE" " 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); " 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 /// In old tables this node may missing or be empty
String replica_metadata; 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()) if (!replica_metadata_exists || replica_metadata.empty())
{ {
/// We have to check shared node granularity before we create ours. /// We have to check shared node granularity before we create ours.
other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper(); other_replicas_fixed_granularity = checkFixedGranualrityInZookeeper();
ReplicatedMergeTreeTableMetadata current_metadata(*this, metadata_snapshot); 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); checkTableStructure(replica_path, metadata_snapshot);
@ -460,8 +475,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat); current_zookeeper->get(zookeeper_path + "/metadata", &metadata_stat);
metadata_version = metadata_stat.version; metadata_version = metadata_stat.version;
} }
/// Temporary directories contain untinalized results of Merges or Fetches (after forced restart) /// Temporary directories contain uninitialized results of Merges or Fetches (after forced restart),
/// and don't allow to reinitialize them, so delete each of them immediately /// don't allow to reinitialize them, delete each of them immediately.
clearOldTemporaryDirectories(0); clearOldTemporaryDirectories(0);
clearOldWriteAheadLogs(); clearOldWriteAheadLogs();
} }
@ -727,12 +742,13 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada
String replicas_value; String replicas_value;
if (!zookeeper->tryGet(zookeeper_path + "/replicas", replicas_value, &replicas_stat)) 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", throw Exception(ErrorCodes::ALL_REPLICAS_LOST,
zookeeper_path), 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. /// 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. /// 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; Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", ops.emplace_back(zkutil::makeCreateRequest(replica_path, "",
@ -761,20 +777,17 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada
Coordination::Responses responses; Coordination::Responses responses;
code = zookeeper->tryMulti(ops, 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)
{ {
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"); LOG_ERROR(log, "Retrying createReplica(), because some other replicas were created at the same time");
} break;
else if (code == Coordination::Error::ZNONODE) case Coordination::Error::ZNONODE:
{ throw Exception(ErrorCodes::ALL_REPLICAS_LOST, "Table {} was suddenly removed", zookeeper_path);
throw Exception("Table " + zookeeper_path + " was suddenly removed.", ErrorCodes::ALL_REPLICAS_LOST); default:
}
else
{
zkutil::KeeperMultiException::check(code, ops, responses); zkutil::KeeperMultiException::check(code, ops, responses);
} }
} while (code == Coordination::Error::ZBADVERSION); } while (code == Coordination::Error::ZBADVERSION);
@ -1123,6 +1136,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
size_t unexpected_parts_nonnew = 0; size_t unexpected_parts_nonnew = 0;
UInt64 unexpected_parts_nonnew_rows = 0; UInt64 unexpected_parts_nonnew_rows = 0;
UInt64 unexpected_parts_rows = 0; UInt64 unexpected_parts_rows = 0;
for (const auto & part : unexpected_parts) for (const auto & part : unexpected_parts)
{ {
if (part->info.level > 0) if (part->info.level > 0)
@ -1134,20 +1148,17 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
unexpected_parts_rows += part->rows_count; unexpected_parts_rows += part->rows_count;
} }
/// Additional helpful statistics const UInt64 parts_to_fetch_blocks = std::accumulate(parts_to_fetch.cbegin(), parts_to_fetch.cend(), 0,
auto get_blocks_count_in_data_part = [&] (const String & part_name) -> UInt64 [&](UInt64 acc, const String& part_name)
{ {
MergeTreePartInfo part_info; MergeTreePartInfo part_info;
if (MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version)) if (MergeTreePartInfo::tryParsePartName(part_name, &part_info, format_version))
return part_info.getBlocksCount(); return acc + part_info.getBlocksCount();
LOG_ERROR(log, "Unexpected part name: {}", part_name); LOG_ERROR(log, "Unexpected part name: {}", part_name);
return 0; return acc;
}; });
UInt64 parts_to_fetch_blocks = 0;
for (const String & name : parts_to_fetch)
parts_to_fetch_blocks += get_blocks_count_in_data_part(name);
/** We can automatically synchronize data, /** 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) * 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) 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); Transaction transaction(*this);
@ -1512,7 +1523,9 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry)
return true; 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) 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. /// 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 local_parts_in_zk = zookeeper->getChildren(fs::path(replica_path) / "parts");
Strings parts_to_remove_from_zk; Strings parts_to_remove_from_zk;
for (const auto & part : local_parts_in_zk) for (const auto & part : local_parts_in_zk)
{ {
if (active_parts_set.getContainingPart(part).empty()) 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); LOG_WARNING(log, "Source replica does not have part {}. Removing it from ZooKeeper.", part);
} }
} }
tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_from_zk); tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_from_zk);
auto local_active_parts = getDataParts(); auto local_active_parts = getDataParts();
DataPartsVector parts_to_remove_from_working_set; DataPartsVector parts_to_remove_from_working_set;
for (const auto & part : local_active_parts) for (const auto & part : local_active_parts)
{ {
if (active_parts_set.getContainingPart(part->name).empty()) 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) if (getSettings()->detach_old_local_parts_when_cloning_replica)
{ {
auto metadata_snapshot = getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr();
for (const auto & part : parts_to_remove_from_working_set) for (const auto & part : parts_to_remove_from_working_set)
{ {
LOG_INFO(log, "Detaching {}", part->relative_path); 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) for (const String & name : active_parts)
{ {
LogEntry log_entry; LogEntry log_entry;
if (!are_restoring_replica)
log_entry.type = LogEntry::GET_PART; 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.source_replica = "";
log_entry.new_part_name = name; log_entry.new_part_name = name;
log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, 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; Coordination::Stat is_lost_stat;
bool is_new_replica = true; bool is_new_replica = true;
String res; String res;
if (zookeeper->tryGet(fs::path(replica_path) / "is_lost", res, &is_lost_stat)) if (zookeeper->tryGet(fs::path(replica_path) / "is_lost", res, &is_lost_stat))
{ {
if (res == "0") if (res == "0")
@ -3968,6 +4015,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora
MinimalisticDataPartChecksums desired_checksums; MinimalisticDataPartChecksums desired_checksums;
String part_path = fs::path(source_replica_path) / "parts" / part_name; String part_path = fs::path(source_replica_path) / "parts" / part_name;
String part_znode = zookeeper->get(part_path); String part_znode = zookeeper->get(part_path);
if (!part_znode.empty()) if (!part_znode.empty())
desired_checksums = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums(); desired_checksums = ReplicatedMergeTreePartHeader::fromString(part_znode).getChecksums();
else else
@ -5030,6 +5078,59 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St
return true; 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<ASTLiteral>(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) void StorageReplicatedMergeTree::dropPartNoWaitNoThrow(const String & part_name)
{ {
assertNotReadonly(); assertNotReadonly();
@ -6938,8 +7039,10 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition(
zookeeper.get(alter_partition_version_path, &alter_partition_version_stat); zookeeper.get(alter_partition_version_path, &alter_partition_version_stat);
MergeTreePartInfo drop_range_info; 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<EphemeralLockInZooKeeper> delimiting_block_lock; std::optional<EphemeralLockInZooKeeper> delimiting_block_lock;
if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info, 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); LOG_INFO(log, "Will not drop partition {}, it is empty.", partition_id);
@ -6960,23 +7063,31 @@ bool StorageReplicatedMergeTree::dropAllPartsInPartition(
entry.create_time = time(nullptr); entry.create_time = time(nullptr);
Coordination::Requests ops; 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. /// 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 /// 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. /// 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::makeCheckRequest(alter_partition_version_path, alter_partition_version_stat.version));
ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", -1)); ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", -1));
/// Just update version, because merges assignment relies on it /// Just update version, because merges assignment relies on it
ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1));
delimiting_block_lock->getUnlockOps(ops); delimiting_block_lock->getUnlockOps(ops);
if (auto txn = query_context->getZooKeeperMetadataTransaction()) if (auto txn = query_context->getZooKeeperMetadataTransaction())
txn->moveOpsTo(ops); txn->moveOpsTo(ops);
Coordination::Responses responses; Coordination::Responses responses;
Coordination::Error code = zookeeper.tryMulti(ops, responses); Coordination::Error code = zookeeper.tryMulti(ops, responses);
if (code == Coordination::Error::ZOK) if (code == Coordination::Error::ZOK)
delimiting_block_lock->assumeUnlocked(); delimiting_block_lock->assumeUnlocked();
else if (code == Coordination::Error::ZBADVERSION) 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 else
zkutil::KeeperMultiException::check(code, ops, responses); zkutil::KeeperMultiException::check(code, ops, responses);

View File

@ -35,7 +35,7 @@
namespace DB 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: * ZooKeeper is used for the following things:
* - the structure of the table (/metadata, /columns) * - the structure of the table (/metadata, /columns)
@ -57,6 +57,7 @@ namespace DB
* Log - a sequence of entries (LogEntry) about what to do. * Log - a sequence of entries (LogEntry) about what to do.
* Each entry is one of: * Each entry is one of:
* - normal data insertion (GET), * - normal data insertion (GET),
* - data insertion with a possible attach from local data (ATTACH),
* - merge (MERGE), * - merge (MERGE),
* - delete the partition (DROP). * - delete the partition (DROP).
* *
@ -65,10 +66,8 @@ namespace DB
* Despite the name of the "queue", execution can be reordered, if necessary (shouldExecuteLogEntry, executeLogEntry). * 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: * 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); * - 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), * - if the part is corrupt (removePartAndEnqueueFetch) or absent during the check
* actions are put on GET from other replicas; * (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).
* *
* The replica to which INSERT was made in the queue will also have an entry of the GET of this data. * 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. * 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 /// Get best replica having this partition on S3
String getSharedDataReplica(const IMergeTreeDataPart & part) const; 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 /// Get throttler for replicated fetches
ThrottlerPtr getFetchesThrottler() const ThrottlerPtr getFetchesThrottler() const
{ {
@ -253,6 +259,8 @@ public:
} }
private: private:
std::atomic_bool are_restoring_replica {false};
/// Get a sequential consistent view of current parts. /// Get a sequential consistent view of current parts.
ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const; ReplicatedMergeTreeQuorumAddedParts::PartitionIdToMaxBlock getMaxAddedBlocks() const;
@ -332,7 +340,7 @@ private:
Poco::Event partial_shutdown_event {false}; /// Poco::Event::EVENT_MANUALRESET Poco::Event partial_shutdown_event {false}; /// Poco::Event::EVENT_MANUALRESET
/// Limiting parallel fetches per node /// 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 /// Limiting parallel fetches per one table
std::atomic_uint current_table_fetches {0}; std::atomic_uint current_table_fetches {0};
@ -389,7 +397,8 @@ private:
*/ */
bool createTableIfNotExists(const StorageMetadataPtr & metadata_snapshot); 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); void createReplica(const StorageMetadataPtr & metadata_snapshot);

View File

@ -79,7 +79,8 @@ ColumnsDescription ITableFunctionFileLike::getActualTableStructure(ContextPtr co
if (structure.empty()) if (structure.empty())
{ {
assert(getName() == "file" && format == "Distributed"); assert(getName() == "file" && format == "Distributed");
Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context); size_t total_bytes_to_read = 0;
Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read);
if (paths.empty()) if (paths.empty())
throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME);
auto read_stream = StorageDistributedDirectoryMonitor::createStreamFromFile(paths[0]); auto read_stream = StorageDistributedDirectoryMonitor::createStreamFromFile(paths[0]);

View File

@ -377,8 +377,8 @@ class ClickhouseIntegrationTestsRunner:
test_cmd = ' '.join([test for test in sorted(test_names)]) test_cmd = ' '.join([test for test in sorted(test_names)])
parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" 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( 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, _get_deselect_option(self.should_skip_tests()), output_path) 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: with open(log_path, 'w') as log:
logging.info("Executing cmd: %s", cmd) logging.info("Executing cmd: %s", cmd)

View File

@ -29,3 +29,9 @@ def cleanup_environment():
pass pass
yield 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

View File

@ -1,6 +1,7 @@
import os import os
import subprocess as sp import subprocess as sp
import tempfile import tempfile
import logging
from threading import Timer from threading import Timer
@ -105,6 +106,7 @@ class CommandRequest:
stderr = self.stderr_file.read().decode('utf-8', errors='replace') stderr = self.stderr_file.read().decode('utf-8', errors='replace')
if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error:
logging.debug(f"Timed out. Last stdout:{stdout}, stderr:{stderr}")
raise QueryTimeoutExceedException('Client timed out!') raise QueryTimeoutExceedException('Client timed out!')
if (self.process.returncode != 0 or stderr) and not self.ignore_error: if (self.process.returncode != 0 or stderr) and not self.ignore_error:

View File

@ -29,7 +29,6 @@ from dict2xml import dict2xml
from kazoo.client import KazooClient from kazoo.client import KazooClient
from kazoo.exceptions import KazooException from kazoo.exceptions import KazooException
from minio import Minio from minio import Minio
from minio.deleteobjects import DeleteObject
from helpers.test_tools import assert_eq_with_retry from helpers.test_tools import assert_eq_with_retry
import docker import docker
@ -172,6 +171,13 @@ def enable_consistent_hash_plugin(rabbitmq_id):
p.communicate() p.communicate()
return p.returncode == 0 return p.returncode == 0
def get_instances_dir():
if 'INTEGRATION_TESTS_RUN_ID' in os.environ and os.environ['INTEGRATION_TESTS_RUN_ID']:
return '_instances_' + shlex.quote(os.environ['INTEGRATION_TESTS_RUN_ID'])
else:
return '_instances'
class ClickHouseCluster: class ClickHouseCluster:
"""ClickHouse cluster with several instances and (possibly) ZooKeeper. """ClickHouse cluster with several instances and (possibly) ZooKeeper.
@ -203,7 +209,14 @@ class ClickHouseCluster:
project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name 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. # 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.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 and 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') self.docker_logs_path = p.join(self.instances_dir, 'docker.log')
self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME)
self.env_variables = {} self.env_variables = {}
@ -421,7 +434,15 @@ class ClickHouseCluster:
pass pass
def get_docker_handle(self, docker_id): def get_docker_handle(self, docker_id):
exception = None
for i in range(5):
try:
return self.docker_client.containers.get(docker_id) 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): def get_client_cmd(self):
cmd = self.client_bin_path cmd = self.client_bin_path
@ -577,7 +598,7 @@ class ClickHouseCluster:
self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')])
self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name,
'--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]
print("HDFS BASE CMD:{}".format(self.base_hdfs_cmd)) logging.debug("HDFS BASE CMD:{self.base_hdfs_cmd)}")
return self.base_hdfs_cmd return self.base_hdfs_cmd
def setup_kerberized_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): def setup_kerberized_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir):
@ -1217,8 +1238,8 @@ class ClickHouseCluster:
for bucket in buckets: for bucket in buckets:
if minio_client.bucket_exists(bucket): if minio_client.bucket_exists(bucket):
delete_object_list = map( delete_object_list = map(
lambda x: DeleteObject(x.object_name), lambda x: x.object_name,
minio_client.list_objects(bucket, recursive=True), minio_client.list_objects_v2(bucket, recursive=True),
) )
errors = minio_client.remove_objects(bucket, delete_object_list) errors = minio_client.remove_objects(bucket, delete_object_list)
for error in errors: for error in errors:
@ -1468,9 +1489,9 @@ class ClickHouseCluster:
instance.docker_client = self.docker_client instance.docker_client = self.docker_client
instance.ip_address = self.get_instance_ip(instance.name) instance.ip_address = self.get_instance_ip(instance.name)
logging.debug("Waiting for ClickHouse start...") logging.debug("Waiting for ClickHouse start in {instance}, ip: {instance.ip_address}...")
instance.wait_for_start(start_timeout) instance.wait_for_start(start_timeout)
logging.debug("ClickHouse started") logging.debug("ClickHouse {instance} started")
instance.client = Client(instance.ip_address, command=self.client_bin_path) instance.client = Client(instance.ip_address, command=self.client_bin_path)
@ -1864,8 +1885,7 @@ class ClickHouseInstance:
self.start_clickhouse(stop_start_wait_sec) self.start_clickhouse(stop_start_wait_sec)
def exec_in_container(self, cmd, detach=False, nothrow=False, **kwargs): def exec_in_container(self, cmd, detach=False, nothrow=False, **kwargs):
container_id = self.get_docker_handle().id return self.cluster.exec_in_container(self.docker_id, cmd, detach, nothrow, **kwargs)
return self.cluster.exec_in_container(container_id, cmd, detach, nothrow, **kwargs)
def contains_in_log(self, substring): def contains_in_log(self, substring):
result = self.exec_in_container( result = self.exec_in_container(
@ -1905,8 +1925,7 @@ class ClickHouseInstance:
["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n' ["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n'
def copy_file_to_container(self, local_path, dest_path): def copy_file_to_container(self, local_path, dest_path):
container_id = self.get_docker_handle().id return self.cluster.copy_file_to_container(self.docker_id, local_path, dest_path)
return self.cluster.copy_file_to_container(container_id, local_path, dest_path)
def get_process_pid(self, process_name): def get_process_pid(self, process_name):
output = self.exec_in_container(["bash", "-c", output = self.exec_in_container(["bash", "-c",
@ -1961,6 +1980,7 @@ class ClickHouseInstance:
self.get_docker_handle().start() self.get_docker_handle().start()
def wait_for_start(self, start_timeout=None, connection_timeout=None): def wait_for_start(self, start_timeout=None, connection_timeout=None):
handle = self.get_docker_handle()
if start_timeout is None or start_timeout <= 0: if start_timeout is None or start_timeout <= 0:
raise Exception("Invalid timeout: {}".format(start_timeout)) raise Exception("Invalid timeout: {}".format(start_timeout))
@ -1983,11 +2003,10 @@ class ClickHouseInstance:
return False return False
while True: while True:
handle = self.get_docker_handle() handle.reload()
status = handle.status status = handle.status
if status == 'exited': if status == 'exited':
raise Exception("Instance `{}' failed to start. Container status: {}, logs: {}" raise Exception(f"Instance `{self.name}' failed to start. Container status: {status}, logs: {handle.logs().decode('utf-8')}")
.format(self.name, status, handle.logs().decode('utf-8')))
deadline = start_time + start_timeout deadline = start_time + start_timeout
# It is possible that server starts slowly. # It is possible that server starts slowly.
@ -1997,9 +2016,8 @@ class ClickHouseInstance:
current_time = time.time() current_time = time.time()
if current_time >= deadline: if current_time >= deadline:
raise Exception("Timed out while waiting for instance `{}' with ip address {} to start. " raise Exception(f"Timed out while waiting for instance `{self.name}' with ip address {self.ip_address} to start. " \
"Container status: {}, logs: {}".format(self.name, self.ip_address, status, f"Container status: {status}, logs: {handle.logs().decode('utf-8')}")
handle.logs().decode('utf-8')))
socket_timeout = min(start_timeout, deadline - current_time) socket_timeout = min(start_timeout, deadline - current_time)

View File

@ -1,5 +1,6 @@
import difflib import difflib
import time import time
import logging
from io import IOBase from io import IOBase
@ -56,7 +57,7 @@ def assert_eq_with_retry(instance, query, expectation, retry_count=20, sleep_tim
break break
time.sleep(sleep_time) time.sleep(sleep_time)
except Exception as ex: except Exception as ex:
print(("assert_eq_with_retry retry {} exception {}".format(i + 1, ex))) logging.exception(f"assert_eq_with_retry retry {i+1} exception {ex}")
time.sleep(sleep_time) time.sleep(sleep_time)
else: else:
val = TSV(get_result(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings, val = TSV(get_result(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings,
@ -76,7 +77,7 @@ def assert_logs_contain_with_retry(instance, substring, retry_count=20, sleep_ti
break break
time.sleep(sleep_time) time.sleep(sleep_time)
except Exception as ex: except Exception as ex:
print("contains_in_log_with_retry retry {} exception {}".format(i + 1, ex)) logging.exception(f"contains_in_log_with_retry retry {i+1} exception {ex}")
time.sleep(sleep_time) time.sleep(sleep_time)
else: else:
raise AssertionError("'{}' not found in logs".format(substring)) raise AssertionError("'{}' not found in logs".format(substring))
@ -89,7 +90,7 @@ def exec_query_with_retry(instance, query, retry_count=40, sleep_time=0.5, setti
break break
except Exception as ex: except Exception as ex:
exception = ex exception = ex
print("Failed to execute query '", query, "' on instance", instance.name, "will retry") logging.exception(f"Failed to execute query '{query}' on instance '{instance.name}' will retry")
time.sleep(sleep_time) time.sleep(sleep_time)
else: else:
raise exception raise exception

View File

@ -1,6 +1,6 @@
[pytest] [pytest]
python_files = test*.py python_files = test*.py
norecursedirs = _instances norecursedirs = _instances*
timeout = 1800 timeout = 1800
junit_duration_report = call junit_duration_report = call
junit_suite_name = integration junit_suite_name = integration

Some files were not shown because too many files have changed in this diff Show More