Merge branch 'master' of github.com:ClickHouse/ClickHouse into en-docs/CLICKHOUSEDOCS-466-query-profiler-settings

This commit is contained in:
Sergei Shtykov 2019-12-05 19:54:00 +03:00
commit 459bfc314b
616 changed files with 11436 additions and 4253 deletions

View File

@ -1,3 +1,82 @@
## ClickHouse release v19.17.4.11, 2019-11-22
### Backward Incompatible Change
* Using column instead of AST to store scalar subquery results for better performance. Setting `enable_scalar_subquery_optimization` was added in 19.17 and it was enabled by default. It leads to errors like [this](https://github.com/ClickHouse/ClickHouse/issues/7851) during upgrade to 19.17.2 or 19.17.3 from previous versions. This setting was disabled by default in 19.17.4, to make possible upgrading from 19.16 and older versions without errors. [#7392](https://github.com/ClickHouse/ClickHouse/pull/7392) ([Amos Bird](https://github.com/amosbird))
### New Feature
* Add the ability to create dictionaries with DDL queries. [#7360](https://github.com/ClickHouse/ClickHouse/pull/7360) ([alesapin](https://github.com/alesapin))
* Make `bloom_filter` type of index supporting `LowCardinality` and `Nullable` [#7363](https://github.com/ClickHouse/ClickHouse/issues/7363) [#7561](https://github.com/ClickHouse/ClickHouse/pull/7561) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
* Add function `isValidJSON` to check that passed string is a valid json. [#5910](https://github.com/ClickHouse/ClickHouse/issues/5910) [#7293](https://github.com/ClickHouse/ClickHouse/pull/7293) ([Vdimir](https://github.com/Vdimir))
* Implement `arrayCompact` function [#7328](https://github.com/ClickHouse/ClickHouse/pull/7328) ([Memo](https://github.com/Joeywzr))
* Created function `hex` for Decimal numbers. It works like `hex(reinterpretAsString())`, but doesn't delete last zero bytes. [#7355](https://github.com/ClickHouse/ClickHouse/pull/7355) ([Mikhail Korotov](https://github.com/millb))
* Add `arrayFill` and `arrayReverseFill` functions, which replace elements by other elements in front/back of them in the array. [#7380](https://github.com/ClickHouse/ClickHouse/pull/7380) ([hcz](https://github.com/hczhcz))
* Add `CRC32IEEE()`/`CRC64()` support [#7480](https://github.com/ClickHouse/ClickHouse/pull/7480) ([Azat Khuzhin](https://github.com/azat))
* Implement `char` function similar to one in [mysql](https://dev.mysql.com/doc/refman/8.0/en/string-functions.html#function_char) [#7486](https://github.com/ClickHouse/ClickHouse/pull/7486) ([sundyli](https://github.com/sundy-li))
* Add `bitmapTransform` function. It transforms an array of values in a bitmap to another array of values, the result is a new bitmap [#7598](https://github.com/ClickHouse/ClickHouse/pull/7598) ([Zhichang Yu](https://github.com/yuzhichang))
* Implemented `javaHashUTF16LE()` function [#7651](https://github.com/ClickHouse/ClickHouse/pull/7651) ([achimbab](https://github.com/achimbab))
* Add `_shard_num` virtual column for the Distributed engine [#7624](https://github.com/ClickHouse/ClickHouse/pull/7624) ([Azat Khuzhin](https://github.com/azat))
### Experimental Feature
* Support for processors (new query execution pipeline) in `MergeTree`. [#7181](https://github.com/ClickHouse/ClickHouse/pull/7181) ([Nikolai Kochetov](https://github.com/KochetovNicolai))
### Bug Fix
* Fix incorrect float parsing in `Values` [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817) [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix))
* Fix rare deadlock which can happen when trace_log is enabled. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov))
* Prevent message duplication when producing Kafka table has any MVs selecting from it [#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7))
* Support for `Array(LowCardinality(Nullable(String)))` in `IN`. Resolves [#7364](https://github.com/ClickHouse/ClickHouse/issues/7364) [#7366](https://github.com/ClickHouse/ClickHouse/pull/7366) ([achimbab](https://github.com/achimbab))
* Add handling of `SQL_TINYINT` and `SQL_BIGINT`, and fix handling of `SQL_FLOAT` data source types in ODBC Bridge. [#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon))
* Fix aggregation (`avg` and quantiles) over empty decimal columns [#7431](https://github.com/ClickHouse/ClickHouse/pull/7431) ([Andrey Konyaev](https://github.com/akonyaev90))
* Fix `INSERT` into Distributed with `MATERIALIZED` columns [#7377](https://github.com/ClickHouse/ClickHouse/pull/7377) ([Azat Khuzhin](https://github.com/azat))
* Make `MOVE PARTITION` work if some parts of partition are already on destination disk or volume [#7434](https://github.com/ClickHouse/ClickHouse/pull/7434) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fixed bug with hardlinks failing to be created during mutations in `ReplicatedMergeTree` in multi-disk configurations. [#7558](https://github.com/ClickHouse/ClickHouse/pull/7558) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fixed a bug with a mutation on a MergeTree when whole part remains unchanged and best space is being found on another disk [#7602](https://github.com/ClickHouse/ClickHouse/pull/7602) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fixed bug with `keep_free_space_ratio` not being read from disks configuration [#7645](https://github.com/ClickHouse/ClickHouse/pull/7645) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fix bug with table contains only `Tuple` columns or columns with complex paths. Fixes [7541](https://github.com/ClickHouse/ClickHouse/issues/7541). [#7545](https://github.com/ClickHouse/ClickHouse/pull/7545) ([alesapin](https://github.com/alesapin))
* Do not account memory for Buffer engine in max_memory_usage limit [#7552](https://github.com/ClickHouse/ClickHouse/pull/7552) ([Azat Khuzhin](https://github.com/azat))
* Fix final mark usage in `MergeTree` tables ordered by `tuple()`. In rare cases it could lead to `Can't adjust last granule` error while select. [#7639](https://github.com/ClickHouse/ClickHouse/pull/7639) ([Anton Popov](https://github.com/CurtizJ))
* Fix bug in mutations that have predicate with actions that require context (for example functions for json), which may lead to crashes or strange exceptions. [#7664](https://github.com/ClickHouse/ClickHouse/pull/7664) ([alesapin](https://github.com/alesapin))
* Fix mismatch of database and table names escaping in `data/` and `shadow/` directories [#7575](https://github.com/ClickHouse/ClickHouse/pull/7575) ([Alexander Burmak](https://github.com/Alex-Burmak))
* Support duplicated keys in RIGHT|FULL JOINs, e.g. ```ON t.x = u.x AND t.x = u.y```. Fix crash in this case. [#7586](https://github.com/ClickHouse/ClickHouse/pull/7586) ([Artem Zuikov](https://github.com/4ertus2))
* Fix `Not found column <expression> in block` when joining on expression with RIGHT or FULL JOIN. [#7641](https://github.com/ClickHouse/ClickHouse/pull/7641) ([Artem Zuikov](https://github.com/4ertus2))
* One more attempt to fix infinite loop in `PrettySpace` format [#7591](https://github.com/ClickHouse/ClickHouse/pull/7591) ([Olga Khvostikova](https://github.com/stavrolia))
* Fix bug in `concat` function when all arguments were `FixedString` of the same size. [#7635](https://github.com/ClickHouse/ClickHouse/pull/7635) ([alesapin](https://github.com/alesapin))
* Fixed exception in case of using 1 argument while defining S3, URL and HDFS storages. [#7618](https://github.com/ClickHouse/ClickHouse/pull/7618) ([Vladimir Chebotarev](https://github.com/excitoon))
* Fix scope of the InterpreterSelectQuery for views with query [#7601](https://github.com/ClickHouse/ClickHouse/pull/7601) ([Azat Khuzhin](https://github.com/azat))
### Improvement
* `Nullable` columns recognized and NULL-values handled correctly by ODBC-bridge [#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk))
* Write current batch for distributed send atomically [#7600](https://github.com/ClickHouse/ClickHouse/pull/7600) ([Azat Khuzhin](https://github.com/azat))
* Throw an exception if we cannot detect table for column name in query. [#7358](https://github.com/ClickHouse/ClickHouse/pull/7358) ([Artem Zuikov](https://github.com/4ertus2))
* Add `merge_max_block_size` setting to `MergeTreeSettings` [#7412](https://github.com/ClickHouse/ClickHouse/pull/7412) ([Artem Zuikov](https://github.com/4ertus2))
* Queries with `HAVING` and without `GROUP BY` assume group by constant. So, `SELECT 1 HAVING 1` now returns a result. [#7496](https://github.com/ClickHouse/ClickHouse/pull/7496) ([Amos Bird](https://github.com/amosbird))
* Support parsing `(X,)` as tuple similar to python. [#7501](https://github.com/ClickHouse/ClickHouse/pull/7501), [#7562](https://github.com/ClickHouse/ClickHouse/pull/7562) ([Amos Bird](https://github.com/amosbird))
* Make `range` function behaviors almost like pythonic one. [#7518](https://github.com/ClickHouse/ClickHouse/pull/7518) ([sundyli](https://github.com/sundy-li))
* Add `constraints` columns to table `system.settings` [#7553](https://github.com/ClickHouse/ClickHouse/pull/7553) ([Vitaly Baranov](https://github.com/vitlibar))
* Better Null format for tcp handler, so that it's possible to use `select ignore(<expression>) from table format Null` for perf measure via clickhouse-client [#7606](https://github.com/ClickHouse/ClickHouse/pull/7606) ([Amos Bird](https://github.com/amosbird))
* Queries like `CREATE TABLE ... AS (SELECT (1, 2))` are parsed correctly [#7542](https://github.com/ClickHouse/ClickHouse/pull/7542) ([hcz](https://github.com/hczhcz))
### Performance Improvement
* The performance of aggregation over short string keys is improved. [#6243](https://github.com/ClickHouse/ClickHouse/pull/6243) ([Alexander Kuzmenkov](https://github.com/akuzm), [Amos Bird](https://github.com/amosbird))
* Run another pass of syntax/expression analysis to get potential optimizations after constant predicates are folded. [#7497](https://github.com/ClickHouse/ClickHouse/pull/7497) ([Amos Bird](https://github.com/amosbird))
* Use storage meta info to evaluate trivial `SELECT count() FROM table;` [#7510](https://github.com/ClickHouse/ClickHouse/pull/7510) ([Amos Bird](https://github.com/amosbird), [alexey-milovidov](https://github.com/alexey-milovidov))
* Vectorize processing `arrayReduce` similar to Aggregator `addBatch`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird))
* Minor improvements in performance of `Kafka` consumption [#7475](https://github.com/ClickHouse/ClickHouse/pull/7475) ([Ivan](https://github.com/abyss7))
### Build/Testing/Packaging Improvement
* Add support for cross-compiling to the CPU architecture AARCH64. Refactor packager script. [#7370](https://github.com/ClickHouse/ClickHouse/pull/7370) [#7539](https://github.com/ClickHouse/ClickHouse/pull/7539) ([Ivan](https://github.com/abyss7))
* Unpack darwin-x86_64 and linux-aarch64 toolchains into mounted Docker volume when building packages [#7534](https://github.com/ClickHouse/ClickHouse/pull/7534) ([Ivan](https://github.com/abyss7))
* Update Docker Image for Binary Packager [#7474](https://github.com/ClickHouse/ClickHouse/pull/7474) ([Ivan](https://github.com/abyss7))
* Fixed compile errors on MacOS Catalina [#7585](https://github.com/ClickHouse/ClickHouse/pull/7585) ([Ernest Poletaev](https://github.com/ernestp))
* Some refactoring in query analysis logic: split complex class into several simple ones. [#7454](https://github.com/ClickHouse/ClickHouse/pull/7454) ([Artem Zuikov](https://github.com/4ertus2))
* Fix build without submodules [#7295](https://github.com/ClickHouse/ClickHouse/pull/7295) ([proller](https://github.com/proller))
* Better `add_globs` in CMake files [#7418](https://github.com/ClickHouse/ClickHouse/pull/7418) ([Amos Bird](https://github.com/amosbird))
* Remove hardcoded paths in `unwind` target [#7460](https://github.com/ClickHouse/ClickHouse/pull/7460) ([Konstantin Podshumok](https://github.com/podshumok))
* Allow to use mysql format without ssl [#7524](https://github.com/ClickHouse/ClickHouse/pull/7524) ([proller](https://github.com/proller))
### Other
* Added ANTLR4 grammar for ClickHouse SQL dialect [#7595](https://github.com/ClickHouse/ClickHouse/issues/7595) [#7596](https://github.com/ClickHouse/ClickHouse/pull/7596) ([alexey-milovidov](https://github.com/alexey-milovidov))
## ClickHouse release v19.16.2.2, 2019-10-30
### Backward Incompatible Change
@ -598,7 +677,7 @@ fix comments to make obvious that it may throw.
### Backward Incompatible Change
* Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov))
* Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_get_any_from_right_table` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2))
* Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_distinct_right_table_keys` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2))
## ClickHouse release 19.13.6.51, 2019-10-02

View File

@ -14,7 +14,4 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Events
* [ClickHouse Meetup in Ankara](https://www.eventbrite.com/e/clickhouse-meetup-ankara-create-blazing-fast-experiences-w-clickhouse-tickets-73100530655) on November 21.
* [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23.
* [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3.
* [ClickHouse Meetup in Moscow](https://yandex.ru/promo/clickhouse/moscow-december-2019) on December 11.

View File

@ -182,6 +182,9 @@ set(SRCS
${HDFS3_SOURCE_DIR}/common/FileWrapper.h
)
# old kernels (< 3.17) doens't have SYS_getrandom. Always use POSIX implementation to have better compatibility
set_source_files_properties(${HDFS3_SOURCE_DIR}/rpc/RpcClient.cpp PROPERTIES COMPILE_FLAGS "-DBOOST_UUID_RANDOM_PROVIDER_FORCE_POSIX=1")
# target
add_library(hdfs3 ${SRCS} ${PROTO_SOURCES} ${PROTO_HEADERS})

2
contrib/libunwind vendored

@ -1 +1 @@
Subproject commit 5afe6d87ae9e66485c7fcb106d2f7c2c0359c8f6
Subproject commit 68cffcbbd1840e14664a5f7f19c5e43f65c525b5

View File

@ -11,7 +11,9 @@ endif ()
set(LIBUNWIND_C_SOURCES
${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1.c
${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1-gcc-ext.c
${LIBUNWIND_SOURCE_DIR}/src/Unwind-sjlj.c)
${LIBUNWIND_SOURCE_DIR}/src/Unwind-sjlj.c
# Use unw_backtrace to override libgcc's backtrace symbol for better ABI compatibility
unwind-override.c)
set_source_files_properties(${LIBUNWIND_C_SOURCES} PROPERTIES COMPILE_FLAGS "-std=c99")
set(LIBUNWIND_ASM_SOURCES

View File

@ -0,0 +1,6 @@
#include <libunwind.h>
int backtrace(void ** buffer, int size)
{
return unw_backtrace(buffer, size);
}

2
contrib/protobuf vendored

@ -1 +1 @@
Subproject commit 12735370922a35f03999afff478e1c6d7aa917a4
Subproject commit 0795fa6bc443666068bec56bf700e1f488f592f1

View File

@ -376,6 +376,10 @@ if (USE_POCO_MONGODB)
dbms_target_link_libraries (PRIVATE ${Poco_MongoDB_LIBRARY})
endif()
if (USE_POCO_REDIS)
dbms_target_link_libraries (PRIVATE ${Poco_Redis_LIBRARY})
endif()
if (USE_POCO_NETSSL)
target_link_libraries (clickhouse_common_io PRIVATE ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY})
dbms_target_link_libraries (PRIVATE ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY})
@ -428,6 +432,8 @@ if (USE_JEMALLOC)
if(NOT MAKE_STATIC_LIBRARIES AND ${JEMALLOC_LIBRARIES} MATCHES "${CMAKE_STATIC_LIBRARY_SUFFIX}$")
# mallctl in dbms/src/Interpreters/AsynchronousMetrics.cpp
# Actually we link JEMALLOC to almost all libraries.
# This is just hotfix for some uninvestigated problem.
target_link_libraries(clickhouse_interpreters PRIVATE ${JEMALLOC_LIBRARIES})
endif()
endif ()

View File

@ -97,8 +97,7 @@
#define BRACK_PASTE_LAST '~'
#define BRACK_PASTE_SLEN 6
/// Make sure we don't get ^J for the enter character.
/// This handler also bypasses some unused macro/event checkings.
/// This handler bypasses some unused macro/event checkings.
static int clickhouse_rl_bracketed_paste_begin(int /* count */, int /* key */)
{
std::string buf;
@ -106,10 +105,10 @@ static int clickhouse_rl_bracketed_paste_begin(int /* count */, int /* key */)
RL_SETSTATE(RL_STATE_MOREINPUT);
SCOPE_EXIT(RL_UNSETSTATE(RL_STATE_MOREINPUT));
char c;
int c;
while ((c = rl_read_key()) >= 0)
{
if (c == '\r' || c == '\n')
if (c == '\r')
c = '\n';
buf.push_back(c);
if (buf.size() >= BRACK_PASTE_SLEN && c == BRACK_PASTE_LAST && buf.substr(buf.size() - BRACK_PASTE_SLEN) == BRACK_PASTE_SUFF)
@ -1112,7 +1111,14 @@ private:
/// Check if server send Exception packet
auto packet_type = connection->checkPacket();
if (packet_type && *packet_type == Protocol::Server::Exception)
{
/*
* We're exiting with error, so it makes sense to kill the
* input stream without waiting for it to complete.
*/
async_block_input->cancel(true);
return;
}
connection->sendData(block);
processed_rows += block.rows();

View File

@ -1,6 +1,7 @@
#include "ClusterCopier.h"
#include <chrono>
#include <optional>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/Logger.h>
#include <Poco/ConsoleChannel.h>
@ -178,7 +179,9 @@ struct ShardPartition
ShardPartition(TaskShard & parent, const String & name_quoted_) : task_shard(parent), name(name_quoted_) {}
String getPartitionPath() const;
String getPartitionCleanStartPath() const;
String getCommonPartitionIsDirtyPath() const;
String getCommonPartitionIsCleanedPath() const;
String getPartitionActiveWorkersPath() const;
String getActiveWorkerPath() const;
String getPartitionShardsPath() const;
@ -259,6 +262,8 @@ struct TaskTable
String getPartitionPath(const String & partition_name) const;
String getPartitionIsDirtyPath(const String & partition_name) const;
String getPartitionIsCleanedPath(const String & partition_name) const;
String getPartitionTaskStatusPath(const String & partition_name) const;
String name_in_config;
@ -369,23 +374,6 @@ struct MultiTransactionInfo
Coordination::Responses responses;
};
/// Atomically checks that is_dirty node is not exists, and made the remaining op
/// Returns relative number of failed operation in the second field (the passed op has 0 index)
static MultiTransactionInfo checkNoNodeAndCommit(
const zkutil::ZooKeeperPtr & zookeeper,
const String & checking_node_path,
Coordination::RequestPtr && op)
{
MultiTransactionInfo info;
info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent));
info.requests.emplace_back(zkutil::makeRemoveRequest(checking_node_path, -1));
info.requests.emplace_back(std::move(op));
info.code = zookeeper->tryMulti(info.requests, info.responses);
return info;
}
// Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key])
std::shared_ptr<ASTStorage> createASTStorageDistributed(
const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr)
@ -431,6 +419,11 @@ String TaskTable::getPartitionPath(const String & partition_name) const
+ "/" + escapeForFileName(partition_name); // 201701
}
String ShardPartition::getPartitionCleanStartPath() const
{
return getPartitionPath() + "/clean_start";
}
String ShardPartition::getPartitionPath() const
{
return task_shard.task_table.getPartitionPath(name);
@ -438,8 +431,9 @@ String ShardPartition::getPartitionPath() const
String ShardPartition::getShardStatusPath() const
{
// /root/table_test.hits/201701/1
return getPartitionPath() + "/shards/" + toString(task_shard.numberInCluster());
// schema: /<root...>/tables/<table>/<partition>/shards/<shard>
// e.g. /root/table_test.hits/201701/shards/1
return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster());
}
String ShardPartition::getPartitionShardsPath() const
@ -462,11 +456,26 @@ String ShardPartition::getCommonPartitionIsDirtyPath() const
return getPartitionPath() + "/is_dirty";
}
String ShardPartition::getCommonPartitionIsCleanedPath() const
{
return getCommonPartitionIsDirtyPath() + "/cleaned";
}
String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const
{
return getPartitionPath(partition_name) + "/is_dirty";
}
String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const
{
return getPartitionIsDirtyPath(partition_name) + "/cleaned";
}
String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const
{
return getPartitionPath(partition_name) + "/shards";
}
String DB::TaskShard::getDescription() const
{
std::stringstream ss;
@ -1129,9 +1138,9 @@ protected:
}
/** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock.
* State of some task could be changed during the processing.
* We have to ensure that all shards have the finished state and there are no dirty flag.
* Moreover, we have to check status twice and check zxid, because state could be changed during the checking.
* State of some task could change during the processing.
* We have to ensure that all shards have the finished state and there is no dirty flag.
* Moreover, we have to check status twice and check zxid, because state can change during the checking.
*/
bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition)
{
@ -1170,11 +1179,23 @@ protected:
}
// Check that partition is not dirty
if (zookeeper->exists(task_table.getPartitionIsDirtyPath(partition_name)))
{
CleanStateClock clean_state_clock (
zookeeper,
task_table.getPartitionIsDirtyPath(partition_name),
task_table.getPartitionIsCleanedPath(partition_name)
);
Coordination::Stat stat;
LogicalClock task_start_clock;
if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat))
task_start_clock = LogicalClock(stat.mzxid);
zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat);
if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid)
{
LOG_INFO(log, "Partition " << partition_name << " become dirty");
return false;
}
}
get_futures.clear();
for (const String & path : status_paths)
@ -1260,17 +1281,135 @@ protected:
return res;
}
bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper)
class LogicalClock
{
public:
std::optional<UInt64> zxid;
LogicalClock() = default;
LogicalClock(UInt64 _zxid)
: zxid(_zxid)
{}
bool hasHappened() const
{
return bool(zxid);
}
// happens-before relation with a reasonable time bound
bool happensBefore(const LogicalClock & other) const
{
const UInt64 HALF = 1ull << 63;
return
!zxid ||
(other.zxid && *zxid <= *other.zxid && *other.zxid - *zxid < HALF) ||
(other.zxid && *zxid >= *other.zxid && *zxid - *other.zxid > HALF);
}
bool operator<=(const LogicalClock & other) const
{
return happensBefore(other);
}
// strict equality check
bool operator==(const LogicalClock & other) const
{
return zxid == other.zxid;
}
};
class CleanStateClock
{
public:
LogicalClock discovery_zxid;
std::optional<UInt32> discovery_version;
LogicalClock clean_state_zxid;
std::optional<UInt32> clean_state_version;
std::shared_ptr<std::atomic_bool> stale;
bool is_clean() const
{
return
!is_stale()
&& (
!discovery_zxid.hasHappened()
|| (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid));
}
bool is_stale() const
{
return stale->load();
}
CleanStateClock(
const zkutil::ZooKeeperPtr & zookeeper,
const String & discovery_path,
const String & clean_state_path)
: stale(std::make_shared<std::atomic_bool>(false))
{
Coordination::Stat stat;
String _some_data;
auto watch_callback =
[stale = stale] (const Coordination::WatchResponse & rsp)
{
auto logger = &Poco::Logger::get("ClusterCopier");
if (rsp.error == Coordination::ZOK)
{
switch (rsp.type)
{
case Coordination::CREATED:
LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path);
stale->store(true);
break;
case Coordination::CHANGED:
LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path);
stale->store(true);
}
}
};
if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback))
{
discovery_zxid = LogicalClock(stat.mzxid);
discovery_version = stat.version;
}
if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback))
{
clean_state_zxid = LogicalClock(stat.mzxid);
clean_state_version = stat.version;
}
}
bool operator==(const CleanStateClock & other) const
{
return !is_stale()
&& !other.is_stale()
&& discovery_zxid == other.discovery_zxid
&& discovery_version == other.discovery_version
&& clean_state_zxid == other.clean_state_zxid
&& clean_state_version == other.clean_state_version;
}
bool operator!=(const CleanStateClock & other) const
{
return !(*this == other);
}
};
bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock)
{
if (is_safe_mode)
throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED);
TaskTable & task_table = task_partition.task_shard.task_table;
String current_shards_path = task_partition.getPartitionShardsPath();
String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath();
String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
String dirt_cleaner_path = is_dirty_flag_path + "/cleaner";
const String current_shards_path = task_partition.getPartitionShardsPath();
const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath();
const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
const String dirt_cleaner_path = is_dirty_flag_path + "/cleaner";
const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath();
zkutil::EphemeralNodeHolder::Ptr cleaner_holder;
try
@ -1294,14 +1433,47 @@ protected:
{
if (stat.numChildren != 0)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers, sleep");
LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep.");
std::this_thread::sleep_for(default_sleep_time);
return false;
}
else
{
zookeeper->remove(current_partition_active_workers_dir);
}
}
{
zkutil::EphemeralNodeHolder::Ptr active_workers_lock;
try
{
active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id);
}
catch (const Coordination::Exception & e)
{
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep");
return false;
}
throw;
}
// Lock the dirty flag
zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value());
zookeeper->tryRemove(task_partition.getPartitionCleanStartPath());
CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
/// Remove all status nodes
zookeeper->tryRemoveRecursive(current_shards_path);
{
Strings children;
if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK)
for (const auto & child : children)
{
zookeeper->removeRecursive(current_shards_path + "/" + child);
}
}
String query = "ALTER TABLE " + getQuotedTable(task_table.table_push);
query += " DROP PARTITION " + task_partition.name + "";
@ -1325,13 +1497,28 @@ protected:
return false;
}
/// Remove the locking node
Coordination::Requests requests;
requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1));
requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1));
zookeeper->multi(requests);
/// Update the locking node
if (!my_clock.is_stale())
{
zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value());
if (my_clock.clean_state_version)
zookeeper->set(is_dirt_cleaned_path, host_id, my_clock.clean_state_version.value());
else
zookeeper->create(is_dirt_cleaned_path, host_id, zkutil::CreateMode::Persistent);
}
else
{
LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing");
/// clean state is stale
return false;
}
LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name);
if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS)
zookeeper->set(current_shards_path, host_id);
}
LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now.");
return true;
}
@ -1362,6 +1549,7 @@ protected:
/// Process each source shard having current partition and copy current partition
/// NOTE: shards are sorted by "distance" to current host
bool has_shard_to_process = false;
for (const TaskShardPtr & shard : task_table.all_shards)
{
/// Does shard have a node with current partition?
@ -1405,6 +1593,7 @@ protected:
bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote;
PartitionTaskStatus task_status = PartitionTaskStatus::Error;
bool was_error = false;
has_shard_to_process = true;
for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num)
{
task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task);
@ -1432,11 +1621,13 @@ protected:
cluster_partition.elapsed_time_seconds += watch.elapsedSeconds();
/// Check that whole cluster partition is done
/// Firstly check number failed partition tasks, than look into ZooKeeper and ensure that each partition is done
/// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done
bool partition_is_done = num_failed_shards == 0;
try
{
partition_is_done = partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards);
partition_is_done =
!has_shard_to_process
|| (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards));
}
catch (...)
{
@ -1526,20 +1717,35 @@ protected:
TaskTable & task_table = task_shard.task_table;
ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name);
/// We need to update table definitions for each partition, it could be changed after ALTER
createShardInternalTables(timeouts, task_shard);
auto zookeeper = context.getZooKeeper();
String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
String current_task_is_active_path = task_partition.getActiveWorkerPath();
String current_task_status_path = task_partition.getShardStatusPath();
const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath();
const String current_task_is_active_path = task_partition.getActiveWorkerPath();
const String current_task_status_path = task_partition.getShardStatusPath();
/// Auxiliary functions:
/// Creates is_dirty node to initialize DROP PARTITION
auto create_is_dirty_node = [&] ()
auto create_is_dirty_node = [&, this] (const CleanStateClock & clock)
{
auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent);
if (code && code != Coordination::ZNODEEXISTS)
throw Coordination::Exception(code, is_dirty_flag_path);
if (clock.is_stale())
LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing");
else if (!clock.is_clean())
LOG_DEBUG(log, "Thank you, Captain Obvious");
else if (clock.discovery_version)
{
LOG_DEBUG(log, "Updating clean state clock");
zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value());
}
else
{
LOG_DEBUG(log, "Creating clean state clock");
zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent);
}
};
/// Returns SELECT query filtering current partition and applying user filter
@ -1563,14 +1769,29 @@ protected:
LOG_DEBUG(log, "Processing " << current_task_status_path);
CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
LogicalClock task_start_clock;
{
Coordination::Stat stat;
if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat))
task_start_clock = LogicalClock(stat.mzxid);
}
/// Do not start if partition is dirty, try to clean it
if (zookeeper->exists(is_dirty_flag_path))
if (clean_state_clock.is_clean()
&& (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock))
{
LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean");
zookeeper->createAncestors(current_task_status_path);
}
else
{
LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it");
try
{
tryDropPartition(task_partition, zookeeper);
tryDropPartition(task_partition, zookeeper, clean_state_clock);
}
catch (...)
{
@ -1598,7 +1819,8 @@ protected:
throw;
}
/// Exit if task has been already processed, create blocking node if it is abandoned
/// Exit if task has been already processed;
/// create blocking node to signal cleaning up if it is abandoned
{
String status_data;
if (zookeeper->tryGet(current_task_status_path, status_data))
@ -1611,21 +1833,21 @@ protected:
}
// Task is abandoned, initialize DROP PARTITION
LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner);
LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner << ". Partition will be dropped and refilled.");
create_is_dirty_node();
create_is_dirty_node(clean_state_clock);
return PartitionTaskStatus::Error;
}
}
zookeeper->createAncestors(current_task_status_path);
/// We need to update table definitions for each partition, it could be changed after ALTER
createShardInternalTables(timeouts, task_shard);
/// Check that destination partition is empty if we are first worker
/// NOTE: this check is incorrect if pull and push tables have different partition key!
String clean_start_status;
if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok")
{
zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), "");
auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", *zookeeper, host_id);
// Maybe we are the first worker
ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()");
UInt64 count;
{
@ -1643,36 +1865,38 @@ protected:
Coordination::Stat stat_shards;
zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards);
/// NOTE: partition is still fresh if dirt discovery happens before cleaning
if (stat_shards.numChildren == 0)
{
LOG_WARNING(log, "There are no any workers for partition " << task_partition.name
LOG_WARNING(log, "There are no workers for partition " << task_partition.name
<< ", but destination table contains " << count << " rows"
<< ". Partition will be dropped and refilled.");
create_is_dirty_node();
create_is_dirty_node(clean_state_clock);
return PartitionTaskStatus::Error;
}
}
zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok");
}
/// At this point, we need to sync that the destination table is clean
/// before any actual work
/// Try start processing, create node about it
{
String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id);
auto op_create = zkutil::makeCreateRequest(current_task_status_path, start_state, zkutil::CreateMode::Persistent);
MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create));
if (info.code)
CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
if (clean_state_clock != new_clean_state_clock)
{
zkutil::KeeperMultiException exception(info.code, info.requests, info.responses);
if (exception.getPathForFirstFailedOp() == is_dirty_flag_path)
{
LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled");
LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing");
return PartitionTaskStatus::Error;
}
throw exception;
else if (!new_clean_state_clock.is_clean())
{
LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled");
create_is_dirty_node(new_clean_state_clock);
return PartitionTaskStatus::Error;
}
zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent);
}
/// Try create table (if not exists) on each shard
@ -1733,12 +1957,13 @@ protected:
output = io_insert.out;
}
/// Fail-fast optimization to abort copying when the current clean state expires
std::future<Coordination::ExistsResponse> future_is_dirty_checker;
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
constexpr UInt64 check_period_milliseconds = 500;
/// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copy data
/// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data
auto cancel_check = [&] ()
{
if (zookeeper->expired())
@ -1754,8 +1979,13 @@ protected:
Coordination::ExistsResponse status = future_is_dirty_checker.get();
if (status.error != Coordination::ZNONODE)
{
LogicalClock dirt_discovery_epoch (status.stat.mzxid);
if (dirt_discovery_epoch == clean_state_clock.discovery_zxid)
return false;
throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
}
}
return false;
};
@ -1789,20 +2019,19 @@ protected:
/// Finalize the processing, change state of current partition task (and also check is_dirty flag)
{
String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
auto op_set = zkutil::makeSetRequest(current_task_status_path, state_finished, 0);
MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set));
if (info.code)
CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path);
if (clean_state_clock != new_clean_state_clock)
{
zkutil::KeeperMultiException exception(info.code, info.requests, info.responses);
if (exception.getPathForFirstFailedOp() == is_dirty_flag_path)
LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled");
else
LOG_INFO(log, "Someone made the node abandoned. Will refill partition. " << zkutil::ZooKeeper::error2string(info.code));
LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing");
return PartitionTaskStatus::Error;
}
else if (!new_clean_state_clock.is_clean())
{
LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled");
create_is_dirty_node(new_clean_state_clock);
return PartitionTaskStatus::Error;
}
zookeeper->set(current_task_status_path, state_finished, 0);
}
LOG_INFO(log, "Partition " << task_partition.name << " copied");

View File

@ -30,6 +30,11 @@ if (Poco_Data_FOUND)
set(CLICKHOUSE_ODBC_BRIDGE_LINK ${CLICKHOUSE_ODBC_BRIDGE_LINK} PRIVATE ${Poco_Data_LIBRARY})
set(CLICKHOUSE_ODBC_BRIDGE_INCLUDE ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE} SYSTEM PRIVATE ${Poco_Data_INCLUDE_DIR})
endif ()
if (USE_JEMALLOC)
# We need to link jemalloc directly to odbc-bridge-library, because in other case
# we will build it with default malloc.
set(CLICKHOUSE_ODBC_BRIDGE_LINK ${CLICKHOUSE_ODBC_BRIDGE_LINK} PRIVATE ${JEMALLOC_LIBRARIES})
endif()
clickhouse_program_add_library(odbc-bridge)

View File

@ -407,16 +407,16 @@ void HTTPHandler::processQuery(
{
if (http_request_compression_method_str == "gzip")
{
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Gzip);
in_post = std::make_unique<ZlibInflatingReadBuffer>(std::move(in_post_raw), CompressionMethod::Gzip);
}
else if (http_request_compression_method_str == "deflate")
{
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Zlib);
in_post = std::make_unique<ZlibInflatingReadBuffer>(std::move(in_post_raw), CompressionMethod::Zlib);
}
#if USE_BROTLI
else if (http_request_compression_method_str == "br")
{
in_post = std::make_unique<BrotliReadBuffer>(*in_post_raw);
in_post = std::make_unique<BrotliReadBuffer>(std::move(in_post_raw));
}
#endif
else

View File

@ -15,6 +15,7 @@
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromPocoSocket.h>
#include <Storages/IStorage.h>
#include <boost/algorithm/string/replace.hpp>
#if USE_POCO_NETSSL
#include <Poco/Net/SecureStreamSocket.h>
@ -267,29 +268,49 @@ void MySQLHandler::comPing()
packet_sender->sendPacket(OK_Packet(0x0, client_capability_flags, 0, 0, 0), true);
}
static bool isFederatedServerSetupCommand(const String & query);
void MySQLHandler::comQuery(ReadBuffer & payload)
{
String query = String(payload.position(), payload.buffer().end());
// This is a workaround in order to support adding ClickHouse to MySQL using federated server.
// As Clickhouse doesn't support these statements, we just send OK packet in response.
if (isFederatedServerSetupCommand(query))
{
packet_sender->sendPacket(OK_Packet(0x00, client_capability_flags, 0, 0, 0), true);
}
else
{
bool with_output = false;
std::function<void(const String &)> set_content_type = [&with_output](const String &) -> void {
with_output = true;
};
const String query("select ''");
ReadBufferFromString empty_select(query);
String replacement_query = "select ''";
bool should_replace = false;
// Translate query from MySQL to ClickHouse.
// This is a temporary workaround until ClickHouse supports the syntax "@@var_name".
if (std::string(payload.position(), payload.buffer().end()) == "select @@version_comment limit 1") // MariaDB client starts session with that query
if (query == "select @@version_comment limit 1") // MariaDB client starts session with that query
{
should_replace = true;
}
// This is a workaround in order to support adding ClickHouse to MySQL using federated server.
if (0 == strncasecmp("SHOW TABLE STATUS LIKE", query.c_str(), 22))
{
should_replace = true;
replacement_query = boost::replace_all_copy(query, "SHOW TABLE STATUS LIKE ", show_table_status_replacement_query);
}
ReadBufferFromString replacement(replacement_query);
Context query_context = connection_context;
executeQuery(should_replace ? empty_select : payload, *out, true, query_context, set_content_type, nullptr);
executeQuery(should_replace ? replacement : payload, *out, true, query_context, set_content_type, nullptr);
if (!with_output)
packet_sender->sendPacket(OK_Packet(0x00, client_capability_flags, 0, 0, 0), true);
}
}
void MySQLHandler::authPluginSSL()
@ -335,4 +356,33 @@ void MySQLHandlerSSL::finishHandshakeSSL(size_t packet_size, char * buf, size_t
#endif
static bool isFederatedServerSetupCommand(const String & query)
{
return 0 == strncasecmp("SET NAMES", query.c_str(), 9) || 0 == strncasecmp("SET character_set_results", query.c_str(), 25)
|| 0 == strncasecmp("SET FOREIGN_KEY_CHECKS", query.c_str(), 22) || 0 == strncasecmp("SET AUTOCOMMIT", query.c_str(), 14)
|| 0 == strncasecmp("SET SESSION TRANSACTION ISOLATION LEVEL", query.c_str(), 39);
}
const String MySQLHandler::show_table_status_replacement_query("SELECT"
" name AS Name,"
" engine AS Engine,"
" '10' AS Version,"
" 'Dynamic' AS Row_format,"
" 0 AS Rows,"
" 0 AS Avg_row_length,"
" 0 AS Data_length,"
" 0 AS Max_data_length,"
" 0 AS Index_length,"
" 0 AS Data_free,"
" 'NULL' AS Auto_increment,"
" metadata_modification_time AS Create_time,"
" metadata_modification_time AS Update_time,"
" metadata_modification_time AS Check_time,"
" 'utf8_bin' AS Collation,"
" 'NULL' AS Checksum,"
" '' AS Create_options,"
" '' AS Comment"
" FROM system.tables"
" WHERE name LIKE ");
}

View File

@ -11,7 +11,6 @@
namespace DB
{
/// Handler for MySQL wire protocol connections. Allows to connect to ClickHouse using MySQL client.
class MySQLHandler : public Poco::Net::TCPServerConnection
{
@ -59,6 +58,9 @@ protected:
std::shared_ptr<WriteBuffer> out;
bool secure_connection = false;
private:
static const String show_table_status_replacement_query;
};
#if USE_SSL && USE_POCO_NETSSL

View File

@ -243,6 +243,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
#endif
global_context->setRemoteHostFilter(config());
std::string path = getCanonicalPath(config().getString("path", DBMS_DEFAULT_PATH));
std::string default_database = config().getString("default_database", "default");
@ -438,6 +440,13 @@ int Server::main(const std::vector<std::string> & /*args*/)
buildLoggers(*config, logger());
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config->has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config->getUInt64("max_table_size_to_drop"));
if (config->has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop"));
},
/* already_loaded = */ true);
@ -469,13 +478,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Limit on total number of concurrently executed queries.
global_context->getProcessList().setMaxSize(config().getInt("max_concurrent_queries", 0));
/// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default)
if (config().has("max_table_size_to_drop"))
global_context->setMaxTableSizeToDrop(config().getUInt64("max_table_size_to_drop"));
if (config().has("max_partition_size_to_drop"))
global_context->setMaxPartitionSizeToDrop(config().getUInt64("max_partition_size_to_drop"));
/// Set up caches.
/// Lower cache size on low-memory systems.

View File

@ -3,6 +3,25 @@
NOTE: User and query level settings are set up in "users.xml" file.
-->
<yandex>
<!-- The list of hosts allowed to use in URL-related storage engines and table functions.
If this section is not present in configuration, all hosts are allowed.
-->
<remote_url_allow_hosts>
<!-- Host should be specified exactly as in URL. The name is checked before DNS resolution.
Example: "yandex.ru", "yandex.ru." and "www.yandex.ru" are different hosts.
If port is explicitly specified in URL, the host:port is checked as a whole.
If host specified here without port, any port with this host allowed.
"yandex.ru" -> "yandex.ru:443", "yandex.ru:80" etc. is allowed, but "yandex.ru:80" -> only "yandex.ru:80" is allowed.
If the host is specified as IP address, it is checked as specified in URL. Example: "[2a02:6b8:a::a]".
If there are redirects and support for redirects is enabled, every redirect (the Location field) is checked.
-->
<!-- Regular expression can be specified. RE2 engine is used for regexps.
Regexps are not aligned: don't forget to add ^ and $. Also don't forget to escape dot (.) metacharacter
(forgetting to do so is a common source of error).
-->
</remote_url_allow_hosts>
<logger>
<!-- Possible levels: https://github.com/pocoproject/poco/blob/develop/Foundation/include/Poco/Logger.h#L105 -->
<level>trace</level>
@ -15,7 +34,6 @@
<!--display_name>production</display_name--> <!-- It is the name that will be shown in the client -->
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<!-- For HTTPS and SSL over native protocol. -->
<!--
<https_port>8443</https_port>
@ -411,7 +429,7 @@
<!-- Protection from accidental DROP.
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
If you want do delete one table and don't want to restart clickhouse-server, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
If you want do delete one table and don't want to change clickhouse-server config, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables.
The same for max_partition_size_to_drop.
Uncomment to disable protection.

View File

@ -105,6 +105,11 @@ public:
return data->getFloat64(0);
}
Float32 getFloat32(size_t) const override
{
return data->getFloat32(0);
}
bool isNullAt(size_t) const override
{
return data->isNullAt(0);
@ -219,6 +224,7 @@ public:
Field getField() const { return getDataColumn()[0]; }
/// The constant value. It is valid even if the size of the column is 0.
template <typename T>
T getValue() const { return getField().safeGet<NearestFieldType<T>>(); }
};

View File

@ -96,6 +96,7 @@ public:
void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast<const Self &>(src).getData()[n]); }
void insertData(const char * pos, size_t /*length*/) override;
void insertDefault() override { data.push_back(T()); }
virtual void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); }
void insert(const Field & x) override { data.push_back(DB::get<NearestFieldType<T>>(x)); }
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
@ -144,7 +145,7 @@ public:
}
void insert(const T value) { data.push_back(value); }
void insertValue(const T value) { data.push_back(value); }
Container & getData() { return data; }
const Container & getData() const { return data; }
const T & getElement(size_t n) const { return data[n]; }

View File

@ -92,6 +92,11 @@ public:
chars.resize_fill(chars.size() + n);
}
virtual void insertManyDefaults(size_t length) override
{
chars.resize_fill(chars.size() + n * length);
}
void popBack(size_t elems) override
{
chars.resize_assume_reserved(chars.size() - n * elems);

View File

@ -59,6 +59,7 @@ public:
UInt64 getUInt(size_t n) const override { return getDictionary().getUInt(getIndexes().getUInt(n)); }
Int64 getInt(size_t n) const override { return getDictionary().getInt(getIndexes().getUInt(n)); }
Float64 getFloat64(size_t n) const override { return getDictionary().getInt(getIndexes().getFloat64(n)); }
Float32 getFloat32(size_t n) const override { return getDictionary().getInt(getIndexes().getFloat32(n)); }
bool getBool(size_t n) const override { return getDictionary().getInt(getIndexes().getBool(n)); }
bool isNullAt(size_t n) const override { return getDictionary().isNullAt(getIndexes().getUInt(n)); }
ColumnPtr cut(size_t start, size_t length) const override

View File

@ -205,6 +205,13 @@ public:
offsets.push_back(offsets.back() + 1);
}
virtual void insertManyDefaults(size_t length) override
{
chars.resize_fill(chars.size() + length);
for (size_t i = 0; i < length; ++i)
offsets.push_back(offsets.back() + 1);
}
int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override
{
const ColumnString & rhs = assert_cast<const ColumnString &>(rhs_);

View File

@ -66,6 +66,7 @@ public:
UInt64 getUInt(size_t n) const override { return getNestedColumn()->getUInt(n); }
Int64 getInt(size_t n) const override { return getNestedColumn()->getInt(n); }
Float64 getFloat64(size_t n) const override { return getNestedColumn()->getFloat64(n); }
Float32 getFloat32(size_t n) const override { return getNestedColumn()->getFloat32(n); }
bool getBool(size_t n) const override { return getNestedColumn()->getBool(n); }
bool isNullAt(size_t n) const override { return is_nullable && n == getNullValueIndex(); }
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;

View File

@ -222,6 +222,12 @@ Float64 ColumnVector<T>::getFloat64(size_t n) const
return static_cast<Float64>(data[n]);
}
template <typename T>
Float32 ColumnVector<T>::getFloat32(size_t n) const
{
return static_cast<Float32>(data[n]);
}
template <typename T>
void ColumnVector<T>::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{

View File

@ -144,6 +144,11 @@ public:
data.push_back(T());
}
virtual void insertManyDefaults(size_t length) override
{
data.resize_fill(data.size() + length, T());
}
void popBack(size_t n) override
{
data.resize_assume_reserved(data.size() - n);
@ -205,6 +210,7 @@ public:
UInt64 get64(size_t n) const override;
Float64 getFloat64(size_t n) const override;
Float32 getFloat32(size_t n) const override;
UInt64 getUInt(size_t n) const override
{

View File

@ -100,6 +100,11 @@ public:
throw Exception("Method getFloat64 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual Float32 getFloat32(size_t /*n*/) const
{
throw Exception("Method getFloat32 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** If column is numeric, return value of n-th element, casted to UInt64.
* For NULL values of Nullable column it is allowed to return arbitrary value.
* Otherwise throw an exception.

View File

@ -18,7 +18,7 @@ static bool sameConstants(const IColumn & a, const IColumn & b)
return assert_cast<const ColumnConst &>(a).getField() == assert_cast<const ColumnConst &>(b).getField();
}
ColumnWithTypeAndName getLeastSuperColumn(std::vector<const ColumnWithTypeAndName *> columns)
ColumnWithTypeAndName getLeastSuperColumn(const std::vector<const ColumnWithTypeAndName *> & columns)
{
if (columns.empty())
throw Exception("Logical error: no src columns for supercolumn", ErrorCodes::LOGICAL_ERROR);

View File

@ -7,6 +7,6 @@ namespace DB
{
/// getLeastSupertype + related column changes
ColumnWithTypeAndName getLeastSuperColumn(std::vector<const ColumnWithTypeAndName *> columns);
ColumnWithTypeAndName getLeastSuperColumn(const std::vector<const ColumnWithTypeAndName *> & columns);
}

View File

@ -464,12 +464,13 @@ namespace ErrorCodes
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY = 487;
extern const int UNKNOWN_DICTIONARY = 488;
extern const int INCORRECT_DICTIONARY_DEFINITION = 489;
extern const int CANNOT_FORMAT_DATETIME = 490;
extern const int UNACCEPTABLE_URL = 491;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
extern const int STD_EXCEPTION = 1001;
extern const int UNKNOWN_EXCEPTION = 1002;
extern const int METRIKA_OTHER_ERROR = 1003;
extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001;
extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002;

View File

@ -261,7 +261,7 @@ std::string getExceptionMessage(const Exception & e, bool with_stacktrace, bool
stream << "Code: " << e.code() << ", e.displayText() = " << text;
if (with_stacktrace && !has_embedded_stack_trace)
stream << ", Stack trace:\n\n" << e.getStackTrace().toString();
stream << ", Stack trace (when copying this message, always include the lines below):\n\n" << e.getStackTrace().toString();
}
catch (...) {}

View File

@ -17,7 +17,6 @@ namespace DB
namespace ErrorCodes
{
extern const int POCO_EXCEPTION;
extern const int METRIKA_OTHER_ERROR;
}
class Exception : public Poco::Exception

View File

@ -84,6 +84,23 @@ struct DefaultHash<T, std::enable_if_t<is_arithmetic_v<T>>>
}
};
template <typename T>
struct DefaultHash<T, std::enable_if_t<DB::IsDecimalNumber<T> && sizeof(T) <= 8>>
{
size_t operator() (T key) const
{
return DefaultHash64<typename T::NativeType>(key);
}
};
template <typename T>
struct DefaultHash<T, std::enable_if_t<DB::IsDecimalNumber<T> && sizeof(T) == 16>>
{
size_t operator() (T key) const
{
return DefaultHash64<Int64>(key >> 64) ^ DefaultHash64<Int64>(key);
}
};
template <typename T> struct HashCRC32;

View File

@ -0,0 +1,62 @@
#include <re2/re2.h>
#include <Common/RemoteHostFilter.h>
#include <Poco/URI.h>
#include <Formats/FormatFactory.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNACCEPTABLE_URL;
}
void RemoteHostFilter::checkURL(const Poco::URI & uri) const
{
if (!checkForDirectEntry(uri.getHost()) &&
!checkForDirectEntry(uri.getHost() + ":" + toString(uri.getPort())))
throw Exception("URL \"" + uri.toString() + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL);
}
void RemoteHostFilter::checkHostAndPort(const std::string & host, const std::string & port) const
{
if (!checkForDirectEntry(host) &&
!checkForDirectEntry(host + ":" + port))
throw Exception("URL \"" + host + ":" + port + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL);
}
void RemoteHostFilter::setValuesFromConfig(const Poco::Util::AbstractConfiguration & config)
{
if (config.has("remote_url_allow_hosts"))
{
std::vector<std::string> keys;
config.keys("remote_url_allow_hosts", keys);
for (auto key : keys)
{
if (startsWith(key, "host_regexp"))
regexp_hosts.push_back(config.getString("remote_url_allow_hosts." + key));
else if (startsWith(key, "host"))
primary_hosts.insert(config.getString("remote_url_allow_hosts." + key));
}
}
}
bool RemoteHostFilter::checkForDirectEntry(const std::string & str) const
{
if (!primary_hosts.empty() || !regexp_hosts.empty())
{
if (primary_hosts.find(str) == primary_hosts.end())
{
for (size_t i = 0; i < regexp_hosts.size(); ++i)
if (re2::RE2::FullMatch(str, regexp_hosts[i]))
return true;
return false;
}
return true;
}
return true;
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <vector>
#include <unordered_set>
#include <Poco/URI.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
class RemoteHostFilter
{
/**
* This class checks if url is allowed.
* If primary_hosts and regexp_hosts are empty all urls are allowed.
*/
public:
void checkURL(const Poco::URI & uri) const; /// If URL not allowed in config.xml throw UNACCEPTABLE_URL Exception
void setValuesFromConfig(const Poco::Util::AbstractConfiguration & config);
void checkHostAndPort(const std::string & host, const std::string & port) const; /// Does the same as checkURL, but for host and port.
private:
std::unordered_set<std::string> primary_hosts; /// Allowed primary (<host>) URL from config.xml
std::vector<std::string> regexp_hosts; /// Allowed regexp (<hots_regexp>) URL from config.xml
bool checkForDirectEntry(const std::string & str) const; /// Checks if the primary_hosts and regexp_hosts contain str. If primary_hosts and regexp_hosts are empty return true.
};
}

View File

@ -158,7 +158,7 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext
break;
}
case SIGPROF:
case SIGTSTP:
{
error << "This is a signal used for debugging purposes by the user.";
break;

View File

@ -100,4 +100,71 @@ size_t getLengthEncodedStringSize(const String & s)
return getLengthEncodedNumberSize(s.size()) + s.size();
}
ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex type_index)
{
ColumnType column_type;
int flags = 0;
switch (type_index)
{
case TypeIndex::UInt8:
column_type = ColumnType::MYSQL_TYPE_TINY;
flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG;
break;
case TypeIndex::UInt16:
column_type = ColumnType::MYSQL_TYPE_SHORT;
flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG;
break;
case TypeIndex::UInt32:
column_type = ColumnType::MYSQL_TYPE_LONG;
flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG;
break;
case TypeIndex::UInt64:
column_type = ColumnType::MYSQL_TYPE_LONGLONG;
flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG;
break;
case TypeIndex::Int8:
column_type = ColumnType::MYSQL_TYPE_TINY;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::Int16:
column_type = ColumnType::MYSQL_TYPE_SHORT;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::Int32:
column_type = ColumnType::MYSQL_TYPE_LONG;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::Int64:
column_type = ColumnType::MYSQL_TYPE_LONGLONG;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::Float32:
column_type = ColumnType::MYSQL_TYPE_FLOAT;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::Float64:
column_type = ColumnType::MYSQL_TYPE_TINY;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::Date:
column_type = ColumnType::MYSQL_TYPE_DATE;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::DateTime:
column_type = ColumnType::MYSQL_TYPE_DATETIME;
flags = ColumnDefinitionFlags::BINARY_FLAG;
break;
case TypeIndex::String:
column_type = ColumnType::MYSQL_TYPE_STRING;
break;
case TypeIndex::FixedString:
column_type = ColumnType::MYSQL_TYPE_STRING;
break;
default:
column_type = ColumnType::MYSQL_TYPE_STRING;
break;
}
return ColumnDefinition(column_name, CharacterSet::binary, 0, column_type, flags, 0);
}
}

View File

@ -130,6 +130,14 @@ enum ColumnType
};
// https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html
enum ColumnDefinitionFlags
{
UNSIGNED_FLAG = 32,
BINARY_FLAG = 128
};
class ProtocolError : public DB::Exception
{
public:
@ -824,19 +832,40 @@ protected:
}
};
ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex index);
namespace ProtocolText
{
class ResultsetRow : public WritePacket
{
std::vector<String> columns;
const Columns & columns;
int row_num;
size_t payload_size = 0;
std::vector<String> serialized;
public:
ResultsetRow() = default;
void appendColumn(String && value)
ResultsetRow(const DataTypes & data_types, const Columns & columns_, int row_num_)
: columns(columns_)
, row_num(row_num_)
{
payload_size += getLengthEncodedStringSize(value);
columns.emplace_back(std::move(value));
for (size_t i = 0; i < columns.size(); i++)
{
if (columns[i]->isNullAt(row_num))
{
payload_size += 1;
serialized.emplace_back("\xfb");
}
else
{
WriteBufferFromOwnString ostr;
data_types[i]->serializeAsText(*columns[i], row_num, ostr, FormatSettings());
payload_size += getLengthEncodedStringSize(ostr.str());
serialized.push_back(std::move(ostr.str()));
}
}
}
protected:
size_t getPayloadSize() const override
{
@ -845,11 +874,18 @@ protected:
void writePayloadImpl(WriteBuffer & buffer) const override
{
for (const String & column : columns)
writeLengthEncodedString(column, buffer);
for (size_t i = 0; i < columns.size(); i++)
{
if (columns[i]->isNullAt(row_num))
buffer.write(serialized[i].data(), 1);
else
writeLengthEncodedString(serialized[i], buffer);
}
}
};
}
namespace Authentication
{

View File

@ -41,8 +41,8 @@ struct Settings : public SettingsCollection<Settings>
* Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field,
* but we are not going to do it, because settings is used everywhere as static struct fields.
*
* `flags` can be either 0 or IGNORABLE.
* A setting is "IGNORABLE" if it doesn't affects the results of the queries and can be ignored without exception.
* `flags` can be either 0 or IMPORTANT.
* A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions.
*/
#define LIST_OF_SETTINGS(M) \
@ -72,10 +72,11 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \
M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \
M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The mininum size of part to upload during multipart upload to S3.", 0) \
M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", 0) \
M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \
M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \
M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \
M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \
M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \
M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.", 0) \
\
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \
@ -90,7 +91,7 @@ struct Settings : public SettingsCollection<Settings>
\
M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \
\
M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = any are present.", 0) \
M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = any are present.", IMPORTANT) \
M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \
\
M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
@ -111,6 +112,9 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \
M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
\
M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \
M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \
\
M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \
M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \
M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.", 0) \
@ -146,7 +150,7 @@ struct Settings : public SettingsCollection<Settings>
\
M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \
\
M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", 0) \
M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \
\
M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \
\
@ -179,7 +183,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \
M(SettingBool, input_format_with_names_use_header, false, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \
M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", 0) \
M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \
M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \
M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \
\
@ -209,10 +213,10 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
\
M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", 0) \
M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \
\
M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \
M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.", 0) \
M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.", IMPORTANT) \
\
M(SettingUInt64, preferred_block_size_bytes, 1000000, "", 0) \
\
@ -307,7 +311,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \
M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \
M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", 0) \
M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \
M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join for LEFT and INNER JOINs.", 0) \
M(SettingBool, partial_merge_join_optimizations, false, "Enable optimizations in partial merge join", 0) \
M(SettingUInt64, default_max_bytes_in_join, 100000000, "Maximum size of right-side table if limit's required but max_bytes_in_join is not set.", 0) \

View File

@ -62,7 +62,7 @@ void SettingNumber<Type>::set(const Field & x)
template <typename Type>
void SettingNumber<Type>::set(const String & x)
{
set(parse<Type>(x));
set(completeParse<Type>(x));
}
template <>

View File

@ -286,7 +286,7 @@ enum class SettingsBinaryFormat
* {
* # define APPLY_FOR_MYSETTINGS(M) \
* M(SettingUInt64, a, 100, "Description of a", 0) \
* M(SettingFloat, f, 3.11, "Description of f", IGNORABLE) // IGNORABLE - means the setting can be ignored by older versions) \
* M(SettingFloat, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \
* M(SettingString, s, "default", "Description of s", 0)
*
* DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS)
@ -316,7 +316,7 @@ private:
StringRef name;
StringRef description;
bool is_ignorable;
bool is_important;
IsChangedFunction is_changed;
GetStringFunction get_string;
GetFieldFunction get_field;

View File

@ -254,7 +254,7 @@ void SettingsCollection<Derived>::serialize(WriteBuffer & buf, SettingsBinaryFor
{
details::SettingsCollectionUtils::serializeName(member.name, buf);
if (format >= SettingsBinaryFormat::STRINGS)
details::SettingsCollectionUtils::serializeFlag(member.is_ignorable, buf);
details::SettingsCollectionUtils::serializeFlag(member.is_important, buf);
member.serialize(castToDerived(), buf, format);
}
}
@ -272,18 +272,20 @@ void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFo
if (name.empty() /* empty string is a marker of the end of settings */)
break;
auto * member = the_members.find(name);
bool is_ignorable = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : false;
bool is_important = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : true;
if (member)
{
member->deserialize(castToDerived(), buf, format);
}
else if (is_ignorable)
else if (is_important)
{
details::SettingsCollectionUtils::throwNameNotFound(name);
}
else
{
details::SettingsCollectionUtils::warningNameNotFound(name);
details::SettingsCollectionUtils::skipValue(buf);
}
else
details::SettingsCollectionUtils::throwNameNotFound(name);
}
}
@ -298,8 +300,8 @@ void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFo
{ \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \
}; \
constexpr int IGNORABLE = 1; \
UNUSED(IGNORABLE); \
constexpr int IMPORTANT = 1; \
UNUSED(IMPORTANT); \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \
} \
/** \
@ -322,7 +324,7 @@ void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFo
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \
FLAGS & IGNORABLE, \
FLAGS & IMPORTANT, \
[](const Derived & d) { return d.NAME.changed; }, \
&Functions::NAME##_getString, &Functions::NAME##_getField, \
&Functions::NAME##_setString, &Functions::NAME##_setField, \

View File

@ -5,6 +5,9 @@
namespace DB
{
using TypeListNumbers = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
using TypeListNativeNumbers = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
using TypeListDecimalNumbers = TypeList<Decimal32, Decimal64, Decimal128>;
using TypeListNumbers = TypeList<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64,
Decimal32, Decimal64, Decimal128>;
}

View File

@ -57,6 +57,13 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server
}
}
void NativeBlockInputStream::resetParser()
{
istr_concrete = nullptr;
use_index = false;
header.clear();
avg_value_size_hints.clear();
}
void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint)
{
@ -159,7 +166,7 @@ Block NativeBlockInputStream::readImpl()
auto & header_column = header.getByName(column.name);
if (!header_column.type->equals(*column.type))
{
column.column = recursiveLowCardinalityConversion(column.column, column.type, header.getByPosition(i).type);
column.column = recursiveTypeConversion(column.column, column.type, header.getByPosition(i).type);
column.type = header.getByPosition(i).type;
}
}

View File

@ -78,6 +78,9 @@ public:
Block getHeader() const override;
void resetParser();
protected:
Block readImpl() override;

View File

@ -0,0 +1,203 @@
#include <DataStreams/ParallelParsingBlockInputStream.h>
#include "ParallelParsingBlockInputStream.h"
namespace DB
{
void ParallelParsingBlockInputStream::segmentatorThreadFunction()
{
setThreadName("Segmentator");
try
{
while (!finished)
{
const auto current_unit_number = segmentator_ticket_number % processing_units.size();
auto & unit = processing_units[current_unit_number];
{
std::unique_lock lock(mutex);
segmentator_condvar.wait(lock,
[&]{ return unit.status == READY_TO_INSERT || finished; });
}
if (finished)
{
break;
}
assert(unit.status == READY_TO_INSERT);
// Segmentating the original input.
unit.segment.resize(0);
const bool have_more_data = file_segmentation_engine(original_buffer,
unit.segment, min_chunk_bytes);
unit.is_last = !have_more_data;
unit.status = READY_TO_PARSE;
scheduleParserThreadForUnitWithNumber(current_unit_number);
++segmentator_ticket_number;
if (!have_more_data)
{
break;
}
}
}
catch (...)
{
onBackgroundException();
}
}
void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_number)
{
try
{
setThreadName("ChunkParser");
auto & unit = processing_units[current_unit_number];
/*
* This is kind of suspicious -- the input_process_creator contract with
* respect to multithreaded use is not clear, but we hope that it is
* just a 'normal' factory class that doesn't have any state, and so we
* can use it from multiple threads simultaneously.
*/
ReadBuffer read_buffer(unit.segment.data(), unit.segment.size(), 0);
auto parser = std::make_unique<InputStreamFromInputFormat>(
input_processor_creator(read_buffer, header, context,
row_input_format_params, format_settings));
unit.block_ext.block.clear();
unit.block_ext.block_missing_values.clear();
// We don't know how many blocks will be. So we have to read them all
// until an empty block occured.
Block block;
while (!finished && (block = parser->read()) != Block())
{
unit.block_ext.block.emplace_back(block);
unit.block_ext.block_missing_values.emplace_back(parser->getMissingValues());
}
// We suppose we will get at least some blocks for a non-empty buffer,
// except at the end of file. Also see a matching assert in readImpl().
assert(unit.is_last || unit.block_ext.block.size() > 0);
std::unique_lock lock(mutex);
unit.status = READY_TO_READ;
reader_condvar.notify_all();
}
catch (...)
{
onBackgroundException();
}
}
void ParallelParsingBlockInputStream::onBackgroundException()
{
tryLogCurrentException(__PRETTY_FUNCTION__);
std::unique_lock lock(mutex);
if (!background_exception)
{
background_exception = std::current_exception();
}
finished = true;
reader_condvar.notify_all();
segmentator_condvar.notify_all();
}
Block ParallelParsingBlockInputStream::readImpl()
{
if (isCancelledOrThrowIfKilled() || finished)
{
/**
* Check for background exception and rethrow it before we return.
*/
std::unique_lock lock(mutex);
if (background_exception)
{
lock.unlock();
cancel(false);
std::rethrow_exception(background_exception);
}
return Block{};
}
const auto current_unit_number = reader_ticket_number % processing_units.size();
auto & unit = processing_units[current_unit_number];
if (!next_block_in_current_unit.has_value())
{
// We have read out all the Blocks from the previous Processing Unit,
// wait for the current one to become ready.
std::unique_lock lock(mutex);
reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || finished; });
if (finished)
{
/**
* Check for background exception and rethrow it before we return.
*/
if (background_exception)
{
lock.unlock();
cancel(false);
std::rethrow_exception(background_exception);
}
return Block{};
}
assert(unit.status == READY_TO_READ);
next_block_in_current_unit = 0;
}
if (unit.block_ext.block.size() == 0)
{
/*
* Can we get zero blocks for an entire segment, when the format parser
* skips it entire content and does not create any blocks? Probably not,
* but if we ever do, we should add a loop around the above if, to skip
* these. Also see a matching assert in the parser thread.
*/
assert(unit.is_last);
finished = true;
return Block{};
}
assert(next_block_in_current_unit.value() < unit.block_ext.block.size());
Block res = std::move(unit.block_ext.block.at(*next_block_in_current_unit));
last_block_missing_values = std::move(unit.block_ext.block_missing_values[*next_block_in_current_unit]);
next_block_in_current_unit.value() += 1;
if (*next_block_in_current_unit == unit.block_ext.block.size())
{
// Finished reading this Processing Unit, move to the next one.
next_block_in_current_unit.reset();
++reader_ticket_number;
if (unit.is_last)
{
// It it was the last unit, we're finished.
finished = true;
}
else
{
// Pass the unit back to the segmentator.
std::unique_lock lock(mutex);
unit.status = READY_TO_INSERT;
segmentator_condvar.notify_all();
}
}
return res;
}
}

View File

@ -0,0 +1,258 @@
#pragma once
#include <DataStreams/IBlockInputStream.h>
#include <Formats/FormatFactory.h>
#include <Common/ThreadPool.h>
#include <Common/setThreadName.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ReadBuffer.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <Interpreters/Context.h>
namespace DB
{
/**
* ORDER-PRESERVING parallel parsing of data formats.
* It splits original data into chunks. Then each chunk is parsed by different thread.
* The number of chunks equals to the number or parser threads.
* The size of chunk is equal to min_chunk_bytes_for_parallel_parsing setting.
*
* This stream has three kinds of threads: one segmentator, multiple parsers,
* and one reader thread -- that is, the one from which readImpl() is called.
* They operate one after another on parts of data called "processing units".
* One unit consists of buffer with raw data from file, filled by segmentator
* thread. This raw data is then parsed by a parser thread to form a number of
* Blocks. These Blocks are returned to the parent stream from readImpl().
* After being read out, a processing unit is reused, to save on allocating
* memory for the raw buffer. The processing units are organized into a circular
* array to facilitate reuse and to apply backpressure on the segmentator thread
* -- after it runs out of processing units, it has to wait for the reader to
* read out the previous blocks.
* The outline of what the threads do is as follows:
* segmentator thread:
* 1) wait for the next processing unit to become empty
* 2) fill it with a part of input file
* 3) start a parser thread
* 4) repeat until eof
* parser thread:
* 1) parse the given raw buffer without any synchronization
* 2) signal that the given unit is ready to read
* 3) finish
* readImpl():
* 1) wait for the next processing unit to become ready to read
* 2) take the blocks from the processing unit to return them to the caller
* 3) signal that the processing unit is empty
* 4) repeat until it encounters unit that is marked as "past_the_end"
* All threads must also check for cancel/eof/exception flags.
*/
class ParallelParsingBlockInputStream : public IBlockInputStream
{
private:
using ReadCallback = std::function<void()>;
using InputProcessorCreator = std::function<InputFormatPtr(
ReadBuffer & buf,
const Block & header,
const Context & context,
const RowInputFormatParams & params,
const FormatSettings & settings)>;
public:
struct InputCreatorParams
{
const Block &sample;
const Context &context;
const RowInputFormatParams& row_input_format_params;
const FormatSettings &settings;
};
struct Params
{
ReadBuffer & read_buffer;
const InputProcessorCreator &input_processor_creator;
const InputCreatorParams &input_creator_params;
FormatFactory::FileSegmentationEngine file_segmentation_engine;
int max_threads;
size_t min_chunk_bytes;
};
explicit ParallelParsingBlockInputStream(const Params & params)
: header(params.input_creator_params.sample),
context(params.input_creator_params.context),
row_input_format_params(params.input_creator_params.row_input_format_params),
format_settings(params.input_creator_params.settings),
input_processor_creator(params.input_processor_creator),
min_chunk_bytes(params.min_chunk_bytes),
original_buffer(params.read_buffer),
// Subtract one thread that we use for segmentation and one for
// reading. After that, must have at least two threads left for
// parsing. See the assertion below.
pool(std::max(2, params.max_threads - 2)),
file_segmentation_engine(params.file_segmentation_engine)
{
// See comment above.
assert(params.max_threads >= 4);
// One unit for each thread, including segmentator and reader, plus a
// couple more units so that the segmentation thread doesn't spuriously
// bump into reader thread on wraparound.
processing_units.resize(params.max_threads + 2);
segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); });
}
String getName() const override { return "ParallelParsing"; }
~ParallelParsingBlockInputStream() override
{
finishAndWait();
}
void cancel(bool kill) override
{
/**
* Can be called multiple times, from different threads. Saturate the
* the kill flag with OR.
*/
if (kill)
is_killed = true;
is_cancelled = true;
/*
* The format parsers themselves are not being cancelled here, so we'll
* have to wait until they process the current block. Given that the
* chunk size is on the order of megabytes, this should't be too long.
* We can't call IInputFormat->cancel here, because the parser object is
* local to the parser thread, and we don't want to introduce any
* synchronization between parser threads and the other threads to get
* better performance. An ideal solution would be to add a callback to
* IInputFormat that checks whether it was cancelled.
*/
finishAndWait();
}
Block getHeader() const override
{
return header;
}
protected:
//Reader routine
Block readImpl() override;
const BlockMissingValues & getMissingValues() const override
{
return last_block_missing_values;
}
private:
const Block header;
const Context context;
const RowInputFormatParams row_input_format_params;
const FormatSettings format_settings;
const InputProcessorCreator input_processor_creator;
const size_t min_chunk_bytes;
/*
* This is declared as atomic to avoid UB, because parser threads access it
* without synchronization.
*/
std::atomic<bool> finished{false};
BlockMissingValues last_block_missing_values;
// Original ReadBuffer to read from.
ReadBuffer & original_buffer;
//Non-atomic because it is used in one thread.
std::optional<size_t> next_block_in_current_unit;
size_t segmentator_ticket_number{0};
size_t reader_ticket_number{0};
std::mutex mutex;
std::condition_variable reader_condvar;
std::condition_variable segmentator_condvar;
// There are multiple "parsers", that's why we use thread pool.
ThreadPool pool;
// Reading and segmentating the file
ThreadFromGlobalPool segmentator_thread;
// Function to segment the file. Then "parsers" will parse that segments.
FormatFactory::FileSegmentationEngine file_segmentation_engine;
enum ProcessingUnitStatus
{
READY_TO_INSERT,
READY_TO_PARSE,
READY_TO_READ
};
struct BlockExt
{
std::vector<Block> block;
std::vector<BlockMissingValues> block_missing_values;
};
struct ProcessingUnit
{
explicit ProcessingUnit()
: status(ProcessingUnitStatus::READY_TO_INSERT)
{
}
BlockExt block_ext;
Memory<> segment;
std::atomic<ProcessingUnitStatus> status;
bool is_last{false};
};
std::exception_ptr background_exception = nullptr;
// We use deque instead of vector, because it does not require a move
// constructor, which is absent for atomics that are inside ProcessingUnit.
std::deque<ProcessingUnit> processing_units;
void scheduleParserThreadForUnitWithNumber(size_t unit_number)
{
pool.scheduleOrThrowOnError(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number));
}
void finishAndWait()
{
finished = true;
{
std::unique_lock lock(mutex);
segmentator_condvar.notify_all();
reader_condvar.notify_all();
}
if (segmentator_thread.joinable())
segmentator_thread.join();
try
{
pool.wait();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void segmentatorThreadFunction();
void parserThreadFunction(size_t bucket_num);
// Save/log a background exception, set termination flag, wake up all
// threads. This function is used by segmentator and parsed threads.
// readImpl() is called from the main thread, so the exception handling
// is different.
void onBackgroundException();
};
};

View File

@ -203,8 +203,15 @@ UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t i
return date_lut.fromDayNum(DayNum(column_date->getData()[ind]));
else if (const ColumnUInt32 * column_date_time = typeid_cast<const ColumnUInt32 *>(column))
return column_date_time->getData()[ind];
else
throw Exception("Unexpected type of result ttl column", ErrorCodes::LOGICAL_ERROR);
else if (const ColumnConst * column_const = typeid_cast<const ColumnConst *>(column))
{
if (typeid_cast<const ColumnUInt16 *>(&column_const->getDataColumn()))
return date_lut.fromDayNum(DayNum(column_const->getValue<UInt16>()));
else if (typeid_cast<const ColumnUInt32 *>(&column_const->getDataColumn()))
return column_const->getValue<UInt32>();
}
throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -894,7 +894,7 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat
if (isColumnedAsNumber(type))
{
MutableColumnUniquePtr column;
TypeListNumbers::forEach(CreateColumnVector(column, *type, creator));
TypeListNativeNumbers::forEach(CreateColumnVector(column, *type, creator));
if (!column)
throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR);

View File

@ -126,6 +126,6 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type);
ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column);
/// Convert column of type from_type to type to_type by converting nested LowCardinality columns.
ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type);
ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type);
}

View File

@ -84,7 +84,7 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column)
return column;
}
ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type)
ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type)
{
if (!column)
return column;
@ -92,10 +92,14 @@ ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const Data
if (from_type->equals(*to_type))
return column;
/// We can allow insert enum column if it's numeric type is the same as the column's type in table.
if (WhichDataType(to_type).isEnum() && from_type->getTypeId() == to_type->getTypeId())
return column;
if (const auto * column_const = typeid_cast<const ColumnConst *>(column.get()))
{
auto & nested = column_const->getDataColumnPtr();
auto nested_no_lc = recursiveLowCardinalityConversion(nested, from_type, to_type);
auto nested_no_lc = recursiveTypeConversion(nested, from_type, to_type);
if (nested.get() == nested_no_lc.get())
return column;
@ -131,7 +135,7 @@ ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const Data
auto & nested_to = to_array_type->getNestedType();
return ColumnArray::create(
recursiveLowCardinalityConversion(column_array->getDataPtr(), nested_from, nested_to),
recursiveTypeConversion(column_array->getDataPtr(), nested_from, nested_to),
column_array->getOffsetsPtr());
}
}
@ -154,7 +158,7 @@ ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const Data
for (size_t i = 0; i < columns.size(); ++i)
{
auto & element = columns[i];
auto element_no_lc = recursiveLowCardinalityConversion(element, from_elements.at(i), to_elements.at(i));
auto element_no_lc = recursiveTypeConversion(element, from_elements.at(i), to_elements.at(i));
if (element.get() != element_no_lc.get())
{
element = element_no_lc;

View File

@ -361,9 +361,8 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
}
catch (const Exception & e)
{
throw Exception("Cannot create table from metadata file " + table_metadata_path + ", error: " + e.displayText() +
", stack trace:\n" + e.getStackTrace().toString(),
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
throw Exception("Cannot create table from metadata file " + table_metadata_path + ". Error: " + DB::getCurrentExceptionMessage(true),
e, DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
}
}

View File

@ -27,6 +27,7 @@
#include <Poco/Event.h>
#include <Common/Stopwatch.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
#include <Common/ThreadPool.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
@ -81,9 +82,8 @@ try
catch (const Exception & e)
{
throw Exception(
"Cannot create object '" + query.table + "' from query " + serializeAST(query) + ", error: " + e.displayText() + ", stack trace:\n"
+ e.getStackTrace().toString(),
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
"Cannot create object '" + query.table + "' from query " + serializeAST(query) + ". Error: " + DB::getCurrentExceptionMessage(true),
e, DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
}
@ -138,8 +138,7 @@ void DatabaseOrdinary::loadStoredObjects(
catch (const Exception & e)
{
throw Exception(
"Cannot parse definition from metadata file " + full_path + ", error: " + e.displayText() + ", stack trace:\n"
+ e.getStackTrace().toString(), ErrorCodes::CANNOT_PARSE_TEXT);
"Cannot parse definition from metadata file " + full_path + ". Error: " + DB::getCurrentExceptionMessage(true), e, ErrorCodes::CANNOT_PARSE_TEXT);
}
});
@ -180,7 +179,15 @@ void DatabaseOrdinary::loadStoredObjects(
auto & external_loader = context.getExternalDictionariesLoader();
external_loader.addConfigRepository(getDatabaseName(), std::move(dictionaries_repository));
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
external_loader.reload(!lazy_load);
auto filter = [this](const std::string & dictionary_name) -> bool
{
if (!startsWith(dictionary_name, name + "." /* db name */))
return false;
LOG_INFO(log, "Loading dictionary " << backQuote(dictionary_name) << ", for database " << backQuote(name));
return true;
};
external_loader.reload(filter, !lazy_load);
}

View File

@ -48,7 +48,7 @@ public:
double getLoadFactor() const override { return static_cast<double>(element_count.load(std::memory_order_relaxed)) / size; }
bool isCached() const override { return true; }
bool supportUpdates() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{

View File

@ -125,7 +125,11 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadAll()
* the necessity of holding process_list_element shared pointer.
*/
if (is_local)
return executeQuery(load_all_query, context, true).in;
{
BlockIO res = executeQuery(load_all_query, context, true);
/// FIXME res.in may implicitly use some objects owned be res, but them will be destructed after return
return res.in;
}
return std::make_shared<RemoteBlockInputStream>(pool, load_all_query, sample_block, context);
}

View File

@ -71,7 +71,7 @@ public:
double getLoadFactor() const override { return static_cast<double>(element_count.load(std::memory_order_relaxed)) / size; }
bool isCached() const override { return true; }
bool supportUpdates() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{

View File

@ -46,8 +46,6 @@ public:
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
bool isCached() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<ComplexKeyHashedDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);

View File

@ -43,8 +43,6 @@ public:
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
bool isCached() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<FlatDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block);

View File

@ -48,8 +48,6 @@ public:
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
bool isCached() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<HashedDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block);

View File

@ -37,8 +37,6 @@ struct IDictionaryBase : public IExternalLoadable
virtual double getLoadFactor() const = 0;
virtual bool isCached() const = 0;
virtual const IDictionarySource * getSource() const = 0;
virtual const DictionaryStructure & getStructure() const = 0;
@ -47,7 +45,7 @@ struct IDictionaryBase : public IExternalLoadable
virtual BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const = 0;
bool supportUpdates() const override { return !isCached(); }
bool supportUpdates() const override { return true; }
bool isModified() const override
{

View File

@ -38,8 +38,6 @@ public:
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
bool isCached() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<RangeHashedDictionary>(dictionary_name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);

View File

@ -47,8 +47,6 @@ public:
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
bool isCached() const override { return false; }
std::shared_ptr<const IExternalLoadable> clone() const override
{
return std::make_shared<TrieDictionary>(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty);

View File

@ -1,8 +1,10 @@
#include <algorithm>
#include <Common/config.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Core/Settings.h>
#include <DataStreams/MaterializingBlockOutputStream.h>
#include <DataStreams/ParallelParsingBlockInputStream.h>
#include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IRowInputFormat.h>
@ -93,7 +95,7 @@ BlockInputStreamPtr FormatFactory::getInput(
if (!getCreators(name).input_processor_creator)
{
const auto & input_getter = getCreators(name).inout_creator;
const auto & input_getter = getCreators(name).input_creator;
if (!input_getter)
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
@ -103,6 +105,37 @@ BlockInputStreamPtr FormatFactory::getInput(
return input_getter(buf, sample, context, max_block_size, callback ? callback : ReadCallback(), format_settings);
}
const Settings & settings = context.getSettingsRef();
const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine;
// Doesn't make sense to use parallel parsing with less than four threads
// (segmentator + two parsers + reader).
if (settings.input_format_parallel_parsing
&& file_segmentation_engine
&& settings.max_threads >= 4)
{
const auto & input_getter = getCreators(name).input_processor_creator;
if (!input_getter)
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
FormatSettings format_settings = getInputFormatSetting(settings);
RowInputFormatParams row_input_format_params;
row_input_format_params.max_block_size = max_block_size;
row_input_format_params.allow_errors_num = format_settings.input_allow_errors_num;
row_input_format_params.allow_errors_ratio = format_settings.input_allow_errors_ratio;
row_input_format_params.callback = std::move(callback);
row_input_format_params.max_execution_time = settings.max_execution_time;
row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode;
auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings};
ParallelParsingBlockInputStream::Params params{buf, input_getter,
input_creator_params, file_segmentation_engine,
static_cast<int>(settings.max_threads),
settings.min_chunk_bytes_for_parallel_parsing};
return std::make_shared<ParallelParsingBlockInputStream>(params);
}
auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback));
return std::make_shared<InputStreamFromInputFormat>(std::move(format));
}
@ -191,7 +224,7 @@ OutputFormatPtr FormatFactory::getOutputFormat(
void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator)
{
auto & target = dict[name].inout_creator;
auto & target = dict[name].input_creator;
if (target)
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(input_creator);
@ -221,6 +254,13 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro
target = std::move(output_creator);
}
void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine)
{
auto & target = dict[name].file_segmentation_engine;
if (target)
throw Exception("FormatFactory: File segmentation engine " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = file_segmentation_engine;
}
/// Formats for both input/output.
@ -241,6 +281,8 @@ void registerInputFormatProcessorTSKV(FormatFactory & factory);
void registerOutputFormatProcessorTSKV(FormatFactory & factory);
void registerInputFormatProcessorJSONEachRow(FormatFactory & factory);
void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory);
void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory);
void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory);
void registerInputFormatProcessorParquet(FormatFactory & factory);
void registerInputFormatProcessorORC(FormatFactory & factory);
void registerOutputFormatProcessorParquet(FormatFactory & factory);
@ -249,6 +291,12 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory);
void registerInputFormatProcessorTemplate(FormatFactory & factory);
void registerOutputFormatProcessorTemplate(FormatFactory &factory);
/// File Segmentation Engines for parallel reading
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory);
void registerFileSegmentationEngineCSV(FormatFactory & factory);
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory);
/// Output only (presentational) formats.
void registerOutputFormatNull(FormatFactory & factory);
@ -290,6 +338,8 @@ FormatFactory::FormatFactory()
registerOutputFormatProcessorTSKV(*this);
registerInputFormatProcessorJSONEachRow(*this);
registerOutputFormatProcessorJSONEachRow(*this);
registerInputFormatProcessorJSONCompactEachRow(*this);
registerOutputFormatProcessorJSONCompactEachRow(*this);
registerInputFormatProcessorProtobuf(*this);
registerOutputFormatProcessorProtobuf(*this);
registerInputFormatProcessorCapnProto(*this);
@ -299,6 +349,9 @@ FormatFactory::FormatFactory()
registerInputFormatProcessorTemplate(*this);
registerOutputFormatProcessorTemplate(*this);
registerFileSegmentationEngineTabSeparated(*this);
registerFileSegmentationEngineCSV(*this);
registerFileSegmentationEngineJSONEachRow(*this);
registerOutputFormatNull(*this);

View File

@ -2,6 +2,7 @@
#include <Core/Types.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <IO/BufferWithOwnMemory.h>
#include <functional>
#include <memory>
@ -41,6 +42,15 @@ public:
/// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer.
using ReadCallback = std::function<void()>;
/** Fast reading data from buffer and save result to memory.
* Reads at least min_chunk_bytes and some more until the end of the chunk, depends on the format.
* Used in ParallelParsingBlockInputStream.
*/
using FileSegmentationEngine = std::function<bool(
ReadBuffer & buf,
DB::Memory<> & memory,
size_t min_chunk_bytes)>;
/// This callback allows to perform some additional actions after writing a single row.
/// It's initial purpose was to flush Kafka message for each row.
using WriteCallback = std::function<void()>;
@ -77,10 +87,11 @@ private:
struct Creators
{
InputCreator inout_creator;
InputCreator input_creator;
OutputCreator output_creator;
InputProcessorCreator input_processor_creator;
OutputProcessorCreator output_processor_creator;
FileSegmentationEngine file_segmentation_engine;
};
using FormatsDictionary = std::unordered_map<String, Creators>;
@ -114,6 +125,7 @@ public:
/// Register format by its name.
void registerInputFormat(const String & name, InputCreator input_creator);
void registerOutputFormat(const String & name, OutputCreator output_creator);
void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine);
void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator);
void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator);

View File

@ -971,8 +971,16 @@ public:
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isStringOrFixedString(arguments[0].type))
{
if (this->getName().find("OrZero") != std::string::npos ||
this->getName().find("OrNull") != std::string::npos)
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() +
". Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (arguments.size() == 2)
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <Core/Types.h>
#include <Common/FieldVisitors.h>
#include "Sources.h"
#include "Sinks.h"
@ -78,10 +79,18 @@ template <typename T>
inline ALWAYS_INLINE void writeSlice(const NumericArraySlice<T> & slice, GenericArraySink & sink)
{
for (size_t i = 0; i < slice.size; ++i)
{
if constexpr (IsDecimalNumber<T>)
{
DecimalField field(T(slice.data[i]), 0); /// TODO: Decimal scale
sink.elements.insert(field);
}
else
{
Field field = T(slice.data[i]);
sink.elements.insert(field);
}
}
sink.current_offset += slice.size;
}
@ -422,8 +431,17 @@ bool sliceHasImpl(const FirstSliceType & first, const SecondSliceType & second,
}
template <typename T, typename U>
bool sliceEqualElements(const NumericArraySlice<T> & first, const NumericArraySlice<U> & second, size_t first_ind, size_t second_ind)
bool sliceEqualElements(const NumericArraySlice<T> & first [[maybe_unused]],
const NumericArraySlice<U> & second [[maybe_unused]],
size_t first_ind [[maybe_unused]],
size_t second_ind [[maybe_unused]])
{
/// TODO: Decimal scale
if constexpr (IsDecimalNumber<T> && IsDecimalNumber<U>)
return accurate::equalsOp(typename T::NativeType(first.data[first_ind]), typename U::NativeType(second.data[second_ind]));
else if constexpr (IsDecimalNumber<T> || IsDecimalNumber<U>)
return false;
else
return accurate::equalsOp(first.data[first_ind], second.data[second_ind]);
}

View File

@ -3,6 +3,7 @@
#include "IArraySink.h"
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
@ -33,17 +34,18 @@ struct NullableValueSource;
template <typename T>
struct NumericArraySink : public ArraySinkImpl<NumericArraySink<T>>
{
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using CompatibleArraySource = NumericArraySource<T>;
using CompatibleValueSource = NumericValueSource<T>;
typename ColumnVector<T>::Container & elements;
typename ColVecType::Container & elements;
typename ColumnArray::Offsets & offsets;
size_t row_num = 0;
ColumnArray::Offset current_offset = 0;
NumericArraySink(ColumnArray & arr, size_t column_size)
: elements(typeid_cast<ColumnVector<T> &>(arr.getData()).getData()), offsets(arr.getOffsets())
: elements(typeid_cast<ColVecType &>(arr.getData()).getData()), offsets(arr.getOffsets())
{
offsets.resize(column_size);
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
@ -30,17 +31,18 @@ namespace GatherUtils
template <typename T>
struct NumericArraySource : public ArraySourceImpl<NumericArraySource<T>>
{
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
using Slice = NumericArraySlice<T>;
using Column = ColumnArray;
const typename ColumnVector<T>::Container & elements;
const typename ColVecType::Container & elements;
const typename ColumnArray::Offsets & offsets;
size_t row_num = 0;
ColumnArray::Offset prev_offset = 0;
explicit NumericArraySource(const ColumnArray & arr)
: elements(typeid_cast<const ColumnVector<T> &>(arr.getData()).getData()), offsets(arr.getOffsets())
: elements(typeid_cast<const ColVecType &>(arr.getData()).getData()), offsets(arr.getOffsets())
{
}
@ -650,7 +652,7 @@ template <typename T>
struct NumericValueSource : ValueSourceImpl<NumericValueSource<T>>
{
using Slice = NumericValueSlice<T>;
using Column = ColumnVector<T>;
using Column = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
const T * begin;
size_t total_rows;

View File

@ -14,7 +14,9 @@ struct ArraySinkCreator<Type, Types...>
{
static std::unique_ptr<IArraySink> create(ColumnArray & col, NullMap * null_map, size_t column_size)
{
if (typeid_cast<ColumnVector<Type> *>(&col.getData()))
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
if (typeid_cast<ColVecType *>(&col.getData()))
{
if (null_map)
return std::make_unique<NullableArraySink<NumericArraySink<Type>>>(col, *null_map, column_size);

View File

@ -14,7 +14,9 @@ struct ArraySourceCreator<Type, Types...>
{
static std::unique_ptr<IArraySource> create(const ColumnArray & col, const NullMap * null_map, bool is_const, size_t total_rows)
{
if (typeid_cast<const ColumnVector<Type> *>(&col.getData()))
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
if (typeid_cast<const ColVecType *>(&col.getData()))
{
if (null_map)
{

View File

@ -14,7 +14,9 @@ struct ValueSourceCreator<Type, Types...>
{
static std::unique_ptr<IValueSource> create(const IColumn & col, const NullMap * null_map, bool is_const, size_t total_rows)
{
if (auto column_vector = typeid_cast<const ColumnVector<Type> *>(&col))
using ColVecType = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
if (auto column_vector = typeid_cast<const ColVecType *>(&col))
{
if (null_map)
{

View File

@ -590,7 +590,7 @@ struct CallPointInPolygon<Type, Types ...>
template <typename PointInPolygonImpl>
static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNumbers>::Type;
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNativeNumbers>::Type;
if (auto column = typeid_cast<const ColumnVector<Type> *>(&x))
return Impl::template call<Type>(*column, y, impl);
return CallPointInPolygon<Types ...>::call(x, y, impl);
@ -616,7 +616,7 @@ struct CallPointInPolygon<>
template <typename PointInPolygonImpl>
ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl)
{
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNumbers>::Type;
using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNativeNumbers>::Type;
return Impl::call(x, y, impl);
}

View File

@ -1,5 +1,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include <Functions/array/FunctionArrayMapped.h>
#include <Functions/FunctionFactory.h>
@ -27,16 +29,23 @@ struct ArrayCompactImpl
template <typename T>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
const ColumnVector<T> * src_values_column = checkAndGetColumn<ColumnVector<T>>(mapped.get());
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
const ColVecType * src_values_column = checkAndGetColumn<ColVecType>(mapped.get());
if (!src_values_column)
return false;
const IColumn::Offsets & src_offsets = array.getOffsets();
const typename ColumnVector<T>::Container & src_values = src_values_column->getData();
const typename ColVecType::Container & src_values = src_values_column->getData();
auto res_values_column = ColumnVector<T>::create(src_values.size());
typename ColumnVector<T>::Container & res_values = res_values_column->getData();
typename ColVecType::MutablePtr res_values_column;
if constexpr (IsDecimalNumber<T>)
res_values_column = ColVecType::create(src_values.size(), src_values.getScale());
else
res_values_column = ColVecType::create(src_values.size());
typename ColVecType::Container & res_values = res_values_column->getData();
size_t src_offsets_size = src_offsets.size();
auto res_offsets_column = ColumnArray::ColumnOffsets::create(src_offsets_size);
IColumn::Offsets & res_offsets = res_offsets_column->getData();
@ -129,7 +138,10 @@ struct ArrayCompactImpl
executeType< Int32 >(mapped, array, res) ||
executeType< Int64 >(mapped, array, res) ||
executeType<Float32>(mapped, array, res) ||
executeType<Float64>(mapped, array, res)))
executeType<Float64>(mapped, array, res)) ||
executeType<Decimal32>(mapped, array, res) ||
executeType<Decimal64>(mapped, array, res) ||
executeType<Decimal128>(mapped, array, res))
{
executeGeneric(mapped, array, res);
}

View File

@ -1,5 +1,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include "FunctionArrayMapped.h"
#include <Functions/FunctionFactory.h>
@ -31,6 +33,13 @@ struct ArrayCumSumImpl
if (which.isFloat())
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>());
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(maxDecimalPrecision<Decimal128>(), scale);
return std::make_shared<DataTypeArray>(nested);
}
throw Exception("arrayCumSum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -38,11 +47,14 @@ struct ArrayCumSumImpl
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
const ColumnVector<Element> * column = checkAndGetColumn<ColumnVector<Element>>(&*mapped);
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<Result>, ColumnDecimal<Result>, ColumnVector<Result>>;
const ColVecType * column = checkAndGetColumn<ColVecType>(&*mapped);
if (!column)
{
const ColumnConst * column_const = checkAndGetColumnConst<ColumnVector<Element>>(&*mapped);
const ColumnConst * column_const = checkAndGetColumnConst<ColVecType>(&*mapped);
if (!column_const)
return false;
@ -50,8 +62,17 @@ struct ArrayCumSumImpl
const Element x = column_const->template getValue<Element>();
const IColumn::Offsets & offsets = array.getOffsets();
auto res_nested = ColumnVector<Result>::create();
typename ColumnVector<Result>::Container & res_values = res_nested->getData();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
{
const typename ColVecType::Container & data =
checkAndGetColumn<ColVecType>(&column_const->getDataColumn())->getData();
res_nested = ColVecResult::create(0, data.getScale());
}
else
res_nested = ColVecResult::create();
typename ColVecResult::Container & res_values = res_nested->getData();
res_values.resize(column_const->size());
size_t pos = 0;
@ -72,11 +93,16 @@ struct ArrayCumSumImpl
return true;
}
const typename ColVecType::Container & data = column->getData();
const IColumn::Offsets & offsets = array.getOffsets();
const typename ColumnVector<Element>::Container & data = column->getData();
auto res_nested = ColumnVector<Result>::create();
typename ColumnVector<Result>::Container & res_values = res_nested->getData();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
res_nested = ColVecResult::create(0, data.getScale());
else
res_nested = ColVecResult::create();
typename ColVecResult::Container & res_values = res_nested->getData();
res_values.resize(data.size());
size_t pos = 0;
@ -110,7 +136,10 @@ struct ArrayCumSumImpl
executeType< Int32, Int64>(mapped, array, res) ||
executeType< Int64, Int64>(mapped, array, res) ||
executeType<Float32,Float64>(mapped, array, res) ||
executeType<Float64,Float64>(mapped, array, res))
executeType<Float64,Float64>(mapped, array, res) ||
executeType<Decimal32, Decimal128>(mapped, array, res) ||
executeType<Decimal64, Decimal128>(mapped, array, res) ||
executeType<Decimal128, Decimal128>(mapped, array, res))
return res;
else
throw Exception("Unexpected column for arrayCumSum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -1,5 +1,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include "FunctionArrayMapped.h"
#include <Functions/FunctionFactory.h>
@ -34,6 +36,13 @@ struct ArrayCumSumNonNegativeImpl
if (which.isFloat())
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>());
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
DataTypePtr nested = std::make_shared<DataTypeDecimal<Decimal128>>(maxDecimalPrecision<Decimal128>(), scale);
return std::make_shared<DataTypeArray>(nested);
}
throw Exception("arrayCumSumNonNegativeImpl cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -41,16 +50,24 @@ struct ArrayCumSumNonNegativeImpl
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
const ColumnVector<Element> * column = checkAndGetColumn<ColumnVector<Element>>(&*mapped);
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<Result>, ColumnDecimal<Result>, ColumnVector<Result>>;
const ColVecType * column = checkAndGetColumn<ColVecType>(&*mapped);
if (!column)
return false;
const IColumn::Offsets & offsets = array.getOffsets();
const typename ColumnVector<Element>::Container & data = column->getData();
const typename ColVecType::Container & data = column->getData();
auto res_nested = ColumnVector<Result>::create();
typename ColumnVector<Result>::Container & res_values = res_nested->getData();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
res_nested = ColVecResult::create(0, data.getScale());
else
res_nested = ColVecResult::create();
typename ColVecResult::Container & res_values = res_nested->getData();
res_values.resize(data.size());
size_t pos = 0;
@ -60,7 +77,7 @@ struct ArrayCumSumNonNegativeImpl
// skip empty arrays
if (pos < offsets[i])
{
accum_sum = data[pos] > 0 ? data[pos] : 0;
accum_sum = data[pos] > 0 ? data[pos] : Element(0);
res_values[pos] = accum_sum;
for (++pos; pos < offsets[i]; ++pos)
{
@ -90,7 +107,10 @@ struct ArrayCumSumNonNegativeImpl
executeType< Int32, Int64>(mapped, array, res) ||
executeType< Int64, Int64>(mapped, array, res) ||
executeType<Float32,Float64>(mapped, array, res) ||
executeType<Float64,Float64>(mapped, array, res))
executeType<Float64,Float64>(mapped, array, res) ||
executeType<Decimal32, Decimal128>(mapped, array, res) ||
executeType<Decimal64, Decimal128>(mapped, array, res) ||
executeType<Decimal128, Decimal128>(mapped, array, res))
return res;
else
throw Exception("Unexpected column for arrayCumSumNonNegativeImpl: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -1,5 +1,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include "FunctionArrayMapped.h"
#include <Functions/FunctionFactory.h>
@ -37,6 +39,9 @@ struct ArrayDifferenceImpl
if (which.isFloat32() || which.isFloat64())
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>());
if (which.isDecimal())
return std::make_shared<DataTypeArray>(expression_return);
throw Exception("arrayDifference cannot process values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
@ -44,16 +49,24 @@ struct ArrayDifferenceImpl
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr)
{
const ColumnVector<Element> * column = checkAndGetColumn<ColumnVector<Element>>(&*mapped);
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<Result>, ColumnDecimal<Result>, ColumnVector<Result>>;
const ColVecType * column = checkAndGetColumn<ColVecType>(&*mapped);
if (!column)
return false;
const IColumn::Offsets & offsets = array.getOffsets();
const typename ColumnVector<Element>::Container & data = column->getData();
const typename ColVecType::Container & data = column->getData();
auto res_nested = ColumnVector<Result>::create();
typename ColumnVector<Result>::Container & res_values = res_nested->getData();
typename ColVecResult::MutablePtr res_nested;
if constexpr (IsDecimalNumber<Element>)
res_nested = ColVecResult::create(0, data.getScale());
else
res_nested = ColVecResult::create();
typename ColVecResult::Container & res_values = res_nested->getData();
res_values.resize(data.size());
size_t pos = 0;
@ -87,7 +100,10 @@ struct ArrayDifferenceImpl
executeType< Int32, Int64>(mapped, array, res) ||
executeType< Int64, Int64>(mapped, array, res) ||
executeType<Float32,Float64>(mapped, array, res) ||
executeType<Float64,Float64>(mapped, array, res))
executeType<Float64,Float64>(mapped, array, res) ||
executeType<Decimal32, Decimal32>(mapped, array, res) ||
executeType<Decimal64, Decimal64>(mapped, array, res) ||
executeType<Decimal128, Decimal128>(mapped, array, res))
return res;
else
throw Exception("Unexpected column for arrayDifference: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -4,6 +4,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h>
@ -12,6 +13,7 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnTuple.h>
#include <Common/HashTable/ClearableHashMap.h>
@ -104,6 +106,19 @@ private:
template <typename T, size_t>
void operator()();
};
struct DecimalExecutor
{
const UnpackedArrays & arrays;
const DataTypePtr & data_type;
ColumnPtr & result;
DecimalExecutor(const UnpackedArrays & arrays_, const DataTypePtr & data_type_, ColumnPtr & result_)
: arrays(arrays_), data_type(data_type_), result(result_) {}
template <typename T, size_t>
void operator()();
};
};
@ -399,7 +414,8 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
ColumnPtr result_column;
auto not_nullable_nested_return_type = removeNullable(nested_return_type);
TypeListNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
TypeListNativeNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column));
TypeListDecimalNumbers::forEach(DecimalExecutor(arrays, not_nullable_nested_return_type, result_column));
using DateMap = ClearableHashMap<DataTypeDate::FieldType, size_t, DefaultHash<DataTypeDate::FieldType>,
HashTableGrower<INITIAL_SIZE_DEGREE>,
@ -445,6 +461,17 @@ void FunctionArrayIntersect::NumberExecutor::operator()()
result = execute<Map, ColumnVector<T>, true>(arrays, ColumnVector<T>::create());
}
template <typename T, size_t>
void FunctionArrayIntersect::DecimalExecutor::operator()()
{
using Map = ClearableHashMap<T, size_t, DefaultHash<T>, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
if (!result)
if (auto * decimal = typeid_cast<const DataTypeDecimal<T> *>(data_type.get()))
result = execute<Map, ColumnDecimal<T>, true>(arrays, ColumnDecimal<T>::create(0, decimal->getScale()));
}
template <typename Map, typename ColumnType, bool is_numeric_column>
ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr)
{

View File

@ -37,10 +37,12 @@ struct ArraySplitImpl
size_t pos = 0;
out_offsets_2.reserve(in_offsets.size()); // the actual size would be equal or larger
out_offsets_2.reserve(in_offsets.size()); // assume the actual size to be equal or larger
out_offsets_1.reserve(in_offsets.size());
for (size_t i = 0; i < in_offsets.size(); ++i)
{
if (pos < in_offsets[i])
{
pos += !reverse;
for (; pos < in_offsets[i] - reverse; ++pos)
@ -51,6 +53,8 @@ struct ArraySplitImpl
pos += reverse;
out_offsets_2.push_back(pos);
}
out_offsets_1.push_back(out_offsets_2.size());
}
}
@ -73,13 +77,21 @@ struct ArraySplitImpl
}
else
{
size_t pos = 0;
out_offsets_2.reserve(in_offsets.size());
out_offsets_1.reserve(in_offsets.size());
for (size_t i = 0; i < in_offsets.size(); ++i)
{
out_offsets_2.push_back(in_offsets[i]);
out_offsets_1.push_back(i + 1);
if (pos < in_offsets[i])
{
pos = in_offsets[i];
out_offsets_2.push_back(pos);
}
out_offsets_1.push_back(out_offsets_2.size());
}
}
}

View File

@ -1,5 +1,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include "FunctionArrayMapped.h"
#include <Functions/FunctionFactory.h>
@ -31,25 +33,43 @@ struct ArraySumImpl
if (which.isFloat())
return std::make_shared<DataTypeFloat64>();
if (which.isDecimal())
{
UInt32 scale = getDecimalScale(*expression_return);
return std::make_shared<DataTypeDecimal<Decimal128>>(maxDecimalPrecision<Decimal128>(), scale);
}
throw Exception("arraySum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
template <typename Element, typename Result>
static bool executeType(const ColumnPtr & mapped, const ColumnArray::Offsets & offsets, ColumnPtr & res_ptr)
{
const ColumnVector<Element> * column = checkAndGetColumn<ColumnVector<Element>>(&*mapped);
using ColVecType = std::conditional_t<IsDecimalNumber<Element>, ColumnDecimal<Element>, ColumnVector<Element>>;
using ColVecResult = std::conditional_t<IsDecimalNumber<Result>, ColumnDecimal<Result>, ColumnVector<Result>>;
const ColVecType * column = checkAndGetColumn<ColVecType>(&*mapped);
if (!column)
{
const ColumnConst * column_const = checkAndGetColumnConst<ColumnVector<Element>>(&*mapped);
const ColumnConst * column_const = checkAndGetColumnConst<ColVecType>(&*mapped);
if (!column_const)
return false;
const Element x = column_const->template getValue<Element>();
auto res_column = ColumnVector<Result>::create(offsets.size());
typename ColumnVector<Result>::Container & res = res_column->getData();
typename ColVecResult::MutablePtr res_column;
if constexpr (IsDecimalNumber<Element>)
{
const typename ColVecType::Container & data =
checkAndGetColumn<ColVecType>(&column_const->getDataColumn())->getData();
res_column = ColVecResult::create(offsets.size(), data.getScale());
}
else
res_column = ColVecResult::create(offsets.size());
typename ColVecResult::Container & res = res_column->getData();
size_t pos = 0;
for (size_t i = 0; i < offsets.size(); ++i)
@ -62,9 +82,15 @@ struct ArraySumImpl
return true;
}
const typename ColumnVector<Element>::Container & data = column->getData();
auto res_column = ColumnVector<Result>::create(offsets.size());
typename ColumnVector<Result>::Container & res = res_column->getData();
const typename ColVecType::Container & data = column->getData();
typename ColVecResult::MutablePtr res_column;
if constexpr (IsDecimalNumber<Element>)
res_column = ColVecResult::create(offsets.size(), data.getScale());
else
res_column = ColVecResult::create(offsets.size());
typename ColVecResult::Container & res = res_column->getData();
size_t pos = 0;
for (size_t i = 0; i < offsets.size(); ++i)
@ -95,7 +121,10 @@ struct ArraySumImpl
executeType< Int32, Int64>(mapped, offsets, res) ||
executeType< Int64, Int64>(mapped, offsets, res) ||
executeType<Float32,Float64>(mapped, offsets, res) ||
executeType<Float64,Float64>(mapped, offsets, res))
executeType<Float64,Float64>(mapped, offsets, res) ||
executeType<Decimal32, Decimal128>(mapped, offsets, res) ||
executeType<Decimal64, Decimal128>(mapped, offsets, res) ||
executeType<Decimal128, Decimal128>(mapped, offsets, res))
return res;
else
throw Exception("Unexpected column for arraySum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);

View File

@ -91,19 +91,7 @@ private:
template <typename T>
static inline void writeNumber2(char * p, T v)
{
static const char digits[201] =
"00010203040506070809"
"10111213141516171819"
"20212223242526272829"
"30313233343536373839"
"40414243444546474849"
"50515253545556575859"
"60616263646566676869"
"70717273747576777879"
"80818283848586878889"
"90919293949596979899";
memcpy(p, &digits[v * 2], 2);
memcpy(p, &digits100[v * 2], 2);
}
template <typename T>

View File

@ -7,12 +7,9 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <ext/range.h>
#include <math.h>
#include <cmath>
#include <array>
#define DEGREES_IN_RADIANS (M_PI / 180.0)
#define EARTH_RADIUS_IN_METERS 6372797.560856
namespace DB
{
@ -24,142 +21,196 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
static inline Float64 degToRad(Float64 angle) { return angle * DEGREES_IN_RADIANS; }
/**
* The function calculates distance in meters between two points on Earth specified by longitude and latitude in degrees.
* The function uses great circle distance formula https://en.wikipedia.org/wiki/Great-circle_distance.
* Throws exception when one or several input values are not within reasonable bounds.
* Latitude must be in [-90, 90], longitude must be [-180, 180]
/** https://en.wikipedia.org/wiki/Great-circle_distance
*
* The function calculates distance in meters between two points on Earth specified by longitude and latitude in degrees.
* The function uses great circle distance formula https://en.wikipedia.org/wiki/Great-circle_distance .
* Throws exception when one or several input values are not within reasonable bounds.
* Latitude must be in [-90, 90], longitude must be [-180, 180].
* Original code of this implementation of this function is here https://github.com/sphinxsearch/sphinx/blob/409f2c2b5b2ff70b04e38f92b6b1a890326bad65/src/sphinxexpr.cpp#L3825.
* Andrey Aksenov, the author of original code, permitted to use this code in ClickHouse under the Apache 2.0 license.
* Presentation about this code from Highload++ Siberia 2019 is here https://github.com/ClickHouse/ClickHouse/files/3324740/1_._._GEODIST_._.pdf
* The main idea of this implementation is optimisations based on Taylor series, trigonometric identity and calculated constants once for cosine, arcsine(sqrt) and look up table.
*/
namespace
{
constexpr double PI = 3.14159265358979323846;
constexpr float TO_RADF = static_cast<float>(PI / 180.0);
constexpr float TO_RADF2 = static_cast<float>(PI / 360.0);
constexpr size_t GEODIST_TABLE_COS = 1024; // maxerr 0.00063%
constexpr size_t GEODIST_TABLE_ASIN = 512;
constexpr size_t GEODIST_TABLE_K = 1024;
float g_GeoCos[GEODIST_TABLE_COS + 1]; /// cos(x) table
float g_GeoAsin[GEODIST_TABLE_ASIN + 1]; /// asin(sqrt(x)) table
float g_GeoFlatK[GEODIST_TABLE_K + 1][2]; /// geodistAdaptive() flat ellipsoid method k1, k2 coeffs table
inline double sqr(double v)
{
return v * v;
}
inline float fsqr(float v)
{
return v * v;
}
void geodistInit()
{
for (size_t i = 0; i <= GEODIST_TABLE_COS; ++i)
g_GeoCos[i] = static_cast<float>(cos(2 * PI * i / GEODIST_TABLE_COS)); // [0, 2 * pi] -> [0, COSTABLE]
for (size_t i = 0; i <= GEODIST_TABLE_ASIN; ++i)
g_GeoAsin[i] = static_cast<float>(asin(
sqrt(static_cast<double>(i) / GEODIST_TABLE_ASIN))); // [0, 1] -> [0, ASINTABLE]
for (size_t i = 0; i <= GEODIST_TABLE_K; ++i)
{
double x = PI * i / GEODIST_TABLE_K - PI * 0.5; // [-pi / 2, pi / 2] -> [0, KTABLE]
g_GeoFlatK[i][0] = static_cast<float>(sqr(111132.09 - 566.05 * cos(2 * x) + 1.20 * cos(4 * x)));
g_GeoFlatK[i][1] = static_cast<float>(sqr(111415.13 * cos(x) - 94.55 * cos(3 * x) + 0.12 * cos(5 * x)));
}
}
inline float geodistDegDiff(float f)
{
f = static_cast<float>(fabs(f));
while (f > 360)
f -= 360;
if (f > 180)
f = 360 - f;
return f;
}
inline float geodistFastCos(float x)
{
float y = static_cast<float>(fabs(x) * GEODIST_TABLE_COS / PI / 2);
int i = static_cast<int>(y);
y -= i;
i &= (GEODIST_TABLE_COS - 1);
return g_GeoCos[i] + (g_GeoCos[i + 1] - g_GeoCos[i]) * y;
}
inline float geodistFastSin(float x)
{
float y = static_cast<float>(fabs(x) * GEODIST_TABLE_COS / PI / 2);
int i = static_cast<int>(y);
y -= i;
i = (i - GEODIST_TABLE_COS / 4) & (GEODIST_TABLE_COS - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2
return g_GeoCos[i] + (g_GeoCos[i + 1] - g_GeoCos[i]) * y;
}
/// fast implementation of asin(sqrt(x))
/// max error in floats 0.00369%, in doubles 0.00072%
inline float geodistFastAsinSqrt(float x)
{
if (x < 0.122)
{
// distance under 4546km, Taylor error under 0.00072%
float y = static_cast<float>(sqrt(x));
return y + x * y * 0.166666666666666f + x * x * y * 0.075f + x * x * x * y * 0.044642857142857f;
}
if (x < 0.948)
{
// distance under 17083km, 512-entry LUT error under 0.00072%
x *= GEODIST_TABLE_ASIN;
int i = static_cast<int>(x);
return g_GeoAsin[i] + (g_GeoAsin[i + 1] - g_GeoAsin[i]) * (x - i);
}
return static_cast<float>(asin(sqrt(x))); // distance over 17083km, just compute honestly
}
}
class FunctionGreatCircleDistance : public IFunction
{
public:
static constexpr auto name = "greatCircleDistance";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionGreatCircleDistance>(); }
private:
enum class instr_type : uint8_t
{
get_float_64,
get_const_float_64
};
using instr_t = std::pair<instr_type, const IColumn *>;
using instrs_t = std::array<instr_t, 4>;
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 4; }
bool useDefaultImplementationForConstants() const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
for (const auto arg_idx : ext::range(0, arguments.size()))
{
const auto arg = arguments[arg_idx].get();
if (!WhichDataType(arg).isFloat64())
if (!WhichDataType(arg).isFloat())
throw Exception(
"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName() + ". Must be Float64",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeFloat64>();
return std::make_shared<DataTypeFloat32>();
}
instrs_t getInstructions(const Block & block, const ColumnNumbers & arguments, bool & out_const)
Float32 greatCircleDistance(Float32 lon1deg, Float32 lat1deg, Float32 lon2deg, Float32 lat2deg)
{
instrs_t result;
out_const = true;
for (const auto arg_idx : ext::range(0, arguments.size()))
if (lon1deg < -180 || lon1deg > 180 ||
lon2deg < -180 || lon2deg > 180 ||
lat1deg < -90 || lat1deg > 90 ||
lat2deg < -90 || lat2deg > 90)
{
const auto column = block.getByPosition(arguments[arg_idx]).column.get();
if (const auto col = checkAndGetColumn<ColumnVector<Float64>>(column))
{
out_const = false;
result[arg_idx] = instr_t{instr_type::get_float_64, col};
throw Exception("Arguments values out of bounds for function " + getName(),
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
else if (const auto col_const = checkAndGetColumnConst<ColumnVector<Float64>>(column))
float lat_diff = geodistDegDiff(lat1deg - lat2deg);
float lon_diff = geodistDegDiff(lon1deg - lon2deg);
if (lon_diff < 13)
{
result[arg_idx] = instr_t{instr_type::get_const_float_64, col_const};
// points are close enough; use flat ellipsoid model
// interpolate sqr(k1), sqr(k2) coefficients using latitudes midpoint
float m = (lat1deg + lat2deg + 180) * GEODIST_TABLE_K / 360; // [-90, 90] degrees -> [0, KTABLE] indexes
size_t i = static_cast<size_t>(m) & (GEODIST_TABLE_K - 1);
float kk1 = g_GeoFlatK[i][0] + (g_GeoFlatK[i + 1][0] - g_GeoFlatK[i][0]) * (m - i);
float kk2 = g_GeoFlatK[i][1] + (g_GeoFlatK[i + 1][1] - g_GeoFlatK[i][1]) * (m - i);
return static_cast<float>(sqrt(kk1 * lat_diff * lat_diff + kk2 * lon_diff * lon_diff));
}
else
throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return result;
}
/// https://en.wikipedia.org/wiki/Great-circle_distance
Float64 greatCircleDistance(Float64 lon1Deg, Float64 lat1Deg, Float64 lon2Deg, Float64 lat2Deg)
{
if (lon1Deg < -180 || lon1Deg > 180 ||
lon2Deg < -180 || lon2Deg > 180 ||
lat1Deg < -90 || lat1Deg > 90 ||
lat2Deg < -90 || lat2Deg > 90)
{
throw Exception("Arguments values out of bounds for function " + getName(), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
// points too far away; use haversine
static const float d = 2 * 6371000;
float a = fsqr(geodistFastSin(lat_diff * TO_RADF2)) +
geodistFastCos(lat1deg * TO_RADF) * geodistFastCos(lat2deg * TO_RADF) *
fsqr(geodistFastSin(lon_diff * TO_RADF2));
return static_cast<float>(d * geodistFastAsinSqrt(a));
}
Float64 lon1Rad = degToRad(lon1Deg);
Float64 lat1Rad = degToRad(lat1Deg);
Float64 lon2Rad = degToRad(lon2Deg);
Float64 lat2Rad = degToRad(lat2Deg);
Float64 u = sin((lat2Rad - lat1Rad) / 2);
Float64 v = sin((lon2Rad - lon1Rad) / 2);
return 2.0 * EARTH_RADIUS_IN_METERS * asin(sqrt(u * u + cos(lat1Rad) * cos(lat2Rad) * v * v));
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const auto size = input_rows_count;
bool result_is_const{};
auto instrs = getInstructions(block, arguments, result_is_const);
if (result_is_const)
{
const auto & colLon1 = assert_cast<const ColumnConst *>(block.getByPosition(arguments[0]).column.get())->getValue<Float64>();
const auto & colLat1 = assert_cast<const ColumnConst *>(block.getByPosition(arguments[1]).column.get())->getValue<Float64>();
const auto & colLon2 = assert_cast<const ColumnConst *>(block.getByPosition(arguments[2]).column.get())->getValue<Float64>();
const auto & colLat2 = assert_cast<const ColumnConst *>(block.getByPosition(arguments[3]).column.get())->getValue<Float64>();
Float64 res = greatCircleDistance(colLon1, colLat1, colLon2, colLat2);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(size, res);
}
else
{
auto dst = ColumnVector<Float64>::create();
auto dst = ColumnVector<Float32>::create();
auto & dst_data = dst->getData();
dst_data.resize(size);
Float64 vals[instrs.size()];
for (const auto row : ext::range(0, size))
{
for (const auto idx : ext::range(0, instrs.size()))
{
if (instr_type::get_float_64 == instrs[idx].first)
vals[idx] = assert_cast<const ColumnVector<Float64> *>(instrs[idx].second)->getData()[row];
else if (instr_type::get_const_float_64 == instrs[idx].first)
vals[idx] = assert_cast<const ColumnConst *>(instrs[idx].second)->getValue<Float64>();
else
throw Exception{"Unknown instruction type in implementation of greatCircleDistance function", ErrorCodes::LOGICAL_ERROR};
}
dst_data[row] = greatCircleDistance(vals[0], vals[1], vals[2], vals[3]);
}
dst_data.resize(input_rows_count);
const IColumn & col_lon1 = *block.getByPosition(arguments[0]).column;
const IColumn & col_lat1 = *block.getByPosition(arguments[1]).column;
const IColumn & col_lon2 = *block.getByPosition(arguments[2]).column;
const IColumn & col_lat2 = *block.getByPosition(arguments[3]).column;
for (size_t row_num = 0; row_num < input_rows_count; ++row_num)
dst_data[row_num] = greatCircleDistance(
col_lon1.getFloat32(row_num), col_lat1.getFloat32(row_num),
col_lon2.getFloat32(row_num), col_lat2.getFloat32(row_num));
block.getByPosition(result).column = std::move(dst);
}
}
};
void registerFunctionGreatCircleDistance(FunctionFactory & factory)
{
geodistInit();
factory.registerFunction<FunctionGreatCircleDistance>();
}

View File

@ -175,9 +175,7 @@ public:
private:
template <typename T0, typename T1>
static constexpr bool allow_arrays =
!IsDecimalNumber<T0> && !IsDecimalNumber<T1> &&
!std::is_same_v<T0, UInt128> && !std::is_same_v<T1, UInt128>;
static constexpr bool allow_arrays = !std::is_same_v<T0, UInt128> && !std::is_same_v<T1, UInt128>;
template <typename T0, typename T1>
static UInt32 decimalScale(Block & block [[maybe_unused]], const ColumnNumbers & arguments [[maybe_unused]])

View File

@ -32,9 +32,9 @@ public:
BrotliDecoderResult result;
};
BrotliReadBuffer::BrotliReadBuffer(ReadBuffer &in_, size_t buf_size, char *existing_memory, size_t alignment)
BrotliReadBuffer::BrotliReadBuffer(std::unique_ptr<ReadBuffer> in_, size_t buf_size, char *existing_memory, size_t alignment)
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment)
, in(in_)
, in(std::move(in_))
, brotli(std::make_unique<BrotliStateWrapper>())
, in_available(0)
, in_data(nullptr)
@ -55,12 +55,12 @@ bool BrotliReadBuffer::nextImpl()
if (!in_available)
{
in.nextIfAtEnd();
in_available = in.buffer().end() - in.position();
in_data = reinterpret_cast<uint8_t *>(in.position());
in->nextIfAtEnd();
in_available = in->buffer().end() - in->position();
in_data = reinterpret_cast<uint8_t *>(in->position());
}
if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in.eof()))
if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in->eof()))
{
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
}
@ -70,12 +70,12 @@ bool BrotliReadBuffer::nextImpl()
brotli->result = BrotliDecoderDecompressStream(brotli->state, &in_available, &in_data, &out_capacity, &out_data, nullptr);
in.position() = in.buffer().end() - in_available;
in->position() = in->buffer().end() - in_available;
working_buffer.resize(internal_buffer.size() - out_capacity);
if (brotli->result == BROTLI_DECODER_RESULT_SUCCESS)
{
if (in.eof())
if (in->eof())
{
eof = true;
return working_buffer.size() != 0;

View File

@ -11,7 +11,7 @@ class BrotliReadBuffer : public BufferWithOwnMemory<ReadBuffer>
{
public:
BrotliReadBuffer(
ReadBuffer & in_,
std::unique_ptr<ReadBuffer> in_,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);
@ -21,7 +21,7 @@ public:
private:
bool nextImpl() override;
ReadBuffer & in;
std::unique_ptr<ReadBuffer> in;
class BrotliStateWrapper;
std::unique_ptr<BrotliStateWrapper> brotli;

View File

@ -12,6 +12,7 @@ enum class CompressionMethod
/// This option corresponds to HTTP Content-Encoding: deflate.
Zlib,
Brotli,
None
};
}

View File

@ -19,7 +19,6 @@ bool PeekableReadBuffer::peekNext()
{
checkStateCorrect();
size_t bytes_read = 0;
Position copy_from = pos;
size_t bytes_to_copy = sub_buf.available();
if (useSubbufferOnly())
@ -27,11 +26,9 @@ bool PeekableReadBuffer::peekNext()
/// Don't have to copy all data from sub-buffer if there is no data in own memory (checkpoint and pos are in sub-buffer)
if (checkpoint)
copy_from = checkpoint;
bytes_read = copy_from - sub_buf.buffer().begin();
bytes_to_copy = sub_buf.buffer().end() - copy_from;
if (!bytes_to_copy)
{
bytes += bytes_read;
sub_buf.position() = copy_from;
/// Both checkpoint and pos are at the end of sub-buffer. Just load next part of data.
@ -50,7 +47,6 @@ bool PeekableReadBuffer::peekNext()
if (useSubbufferOnly())
{
bytes += bytes_read;
sub_buf.position() = copy_from;
}
@ -198,7 +194,6 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append)
/// Move unread data to the beginning of own memory instead of resize own memory
peeked_size -= offset;
memmove(memory.data(), memory.data() + offset, peeked_size);
bytes += offset;
if (need_update_checkpoint)
checkpoint -= offset;

View File

@ -1,6 +1,7 @@
#include <IO/ReadBufferFromS3.h>
#include <IO/ReadBufferFromIStream.h>
#include <IO/S3Common.h>
#include <common/logger_useful.h>
@ -10,14 +11,15 @@ namespace DB
const int DEFAULT_S3_MAX_FOLLOW_GET_REDIRECT = 2;
ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_,
ReadBufferFromS3::ReadBufferFromS3(const Poco::URI & uri_,
const String & access_key_id_,
const String & secret_access_key_,
const ConnectionTimeouts & timeouts,
const Poco::Net::HTTPBasicCredentials & credentials,
size_t buffer_size_)
const RemoteHostFilter & remote_host_filter_)
: ReadBuffer(nullptr, 0)
, uri {uri_}
, method {Poco::Net::HTTPRequest::HTTP_GET}
, session {makeHTTPSession(uri_, timeouts)}
, remote_host_filter {remote_host_filter_}
{
Poco::Net::HTTPResponse response;
std::unique_ptr<Poco::Net::HTTPRequest> request;
@ -28,11 +30,13 @@ ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_,
if (uri.getPath().empty())
uri.setPath("/");
request = std::make_unique<Poco::Net::HTTPRequest>(method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
request = std::make_unique<Poco::Net::HTTPRequest>(
Poco::Net::HTTPRequest::HTTP_GET,
uri.getPathAndQuery(),
Poco::Net::HTTPRequest::HTTP_1_1);
request->setHost(uri.getHost()); // use original, not resolved host name in header
if (!credentials.getUsername().empty())
credentials.authenticate(*request);
S3Helper::authenticateRequest(*request, access_key_id_, secret_access_key_);
LOG_TRACE((&Logger::get("ReadBufferFromS3")), "Sending request to " << uri.toString());
@ -50,11 +54,12 @@ ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_,
break;
uri = location_iterator->second;
remote_host_filter.checkURL(uri);
session = makeHTTPSession(uri, timeouts);
}
assertResponseIsOk(*request, response, *istr);
impl = std::make_unique<ReadBufferFromIStream>(*istr, buffer_size_);
impl = std::make_unique<ReadBufferFromIStream>(*istr, DBMS_DEFAULT_BUFFER_SIZE);
}

View File

@ -17,17 +17,18 @@ class ReadBufferFromS3 : public ReadBuffer
{
protected:
Poco::URI uri;
std::string method;
HTTPSessionPtr session;
std::istream * istr; /// owned by session
std::unique_ptr<ReadBuffer> impl;
RemoteHostFilter remote_host_filter;
public:
explicit ReadBufferFromS3(Poco::URI uri_,
explicit ReadBufferFromS3(const Poco::URI & uri_,
const String & access_key_id_,
const String & secret_access_key_,
const ConnectionTimeouts & timeouts = {},
const Poco::Net::HTTPBasicCredentials & credentials = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE);
const RemoteHostFilter & remote_host_filter_ = {});
bool nextImpl() override;
};

View File

@ -1053,4 +1053,35 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf)
}
}
void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current)
{
assert(current >= in.position());
assert(current <= in.buffer().end());
const int old_bytes = memory.size();
const int additional_bytes = current - in.position();
const int new_bytes = old_bytes + additional_bytes;
/// There are no new bytes to add to memory.
/// No need to do extra stuff.
if (new_bytes == 0)
return;
memory.resize(new_bytes);
memcpy(memory.data() + old_bytes, in.position(), additional_bytes);
in.position() = current;
}
bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current)
{
assert(current <= in.buffer().end());
if (current < in.buffer().end())
return true;
saveUpToPosition(in, memory, current);
bool loaded_more = !in.eof();
assert(in.position() == in.buffer().begin());
current = in.position();
return loaded_more;
}
}

View File

@ -23,9 +23,11 @@
#include <Formats/FormatSettings.h>
#include <IO/CompressionMethod.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/VarInt.h>
#include <IO/ZlibInflatingReadBuffer.h>
#ifdef __clang__
#pragma clang diagnostic push
@ -875,6 +877,30 @@ inline T parse(const char * data, size_t size)
return res;
}
/// Read something from text format, but expect complete parse of given text
/// For example: 723145 -- ok, 213MB -- not ok
template <typename T>
inline T completeParse(const char * data, size_t size)
{
T res;
ReadBufferFromMemory buf(data, size);
readText(res, buf);
assertEOF(buf);
return res;
}
template <typename T>
inline T completeParse(const String & s)
{
return completeParse<T>(s.data(), s.size());
}
template <typename T>
inline T completeParse(const char * data)
{
return completeParse<T>(data, strlen(data));
}
template <typename T>
inline T parse(const char * data)
{
@ -911,4 +937,29 @@ void skipToNextLineOrEOF(ReadBuffer & buf);
/// Skip to next character after next unescaped \n. If no \n in stream, skip to end. Does not throw on invalid escape sequences.
void skipToUnescapedNextLineOrEOF(ReadBuffer & buf);
template <class TReadBuffer, class... Types>
std::unique_ptr<ReadBuffer> getReadBuffer(const DB::CompressionMethod method, Types&&... args)
{
if (method == DB::CompressionMethod::Gzip)
{
auto read_buf = std::make_unique<TReadBuffer>(std::forward<Types>(args)...);
return std::make_unique<ZlibInflatingReadBuffer>(std::move(read_buf), method);
}
return std::make_unique<TReadBuffer>(args...);
}
/** This function just copies the data from buffer's internal position (in.position())
* to current position (from arguments) into memory.
*/
void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current);
/** This function is negative to eof().
* In fact it returns whether the data was loaded to internal ReadBuffers's buffer or not.
* And saves data from buffer's position to current if there is no pending data in buffer.
* Why we have to use this strange function? Consider we have buffer's internal position in the middle
* of our buffer and the current cursor in the end of the buffer. When we call eof() it calls next().
* And this function can fill the buffer with new data, so we will lose the data from previous buffer state.
*/
bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current);
}

View File

@ -101,6 +101,7 @@ namespace detail
const Poco::Net::HTTPBasicCredentials & credentials;
std::vector<Poco::Net::HTTPCookie> cookies;
HTTPHeaderEntries http_header_entries;
RemoteHostFilter remote_host_filter;
std::istream * call(const Poco::URI uri_, Poco::Net::HTTPResponse & response)
{
@ -157,7 +158,8 @@ namespace detail
OutStreamCallback out_stream_callback_ = {},
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
HTTPHeaderEntries http_header_entries_ = {})
HTTPHeaderEntries http_header_entries_ = {},
const RemoteHostFilter & remote_host_filter_ = {})
: ReadBuffer(nullptr, 0)
, uri {uri_}
, method {!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET}
@ -165,6 +167,7 @@ namespace detail
, out_stream_callback {out_stream_callback_}
, credentials {credentials_}
, http_header_entries {http_header_entries_}
, remote_host_filter {remote_host_filter_}
{
Poco::Net::HTTPResponse response;
@ -173,6 +176,7 @@ namespace detail
while (isRedirect(response.getStatus()))
{
Poco::URI uri_redirect(response.get("Location"));
remote_host_filter.checkURL(uri_redirect);
session->updateSession(uri_redirect);
@ -243,8 +247,9 @@ public:
const DB::SettingUInt64 max_redirects = 0,
const Poco::Net::HTTPBasicCredentials & credentials_ = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const HTTPHeaderEntries & http_header_entries_ = {})
: Parent(std::make_shared<UpdatableSession>(uri_, timeouts, max_redirects), uri_, method_, out_stream_callback_, credentials_, buffer_size_, http_header_entries_)
const HTTPHeaderEntries & http_header_entries_ = {},
const RemoteHostFilter & remote_host_filter_ = {})
: Parent(std::make_shared<UpdatableSession>(uri_, timeouts, max_redirects), uri_, method_, out_stream_callback_, credentials_, buffer_size_, http_header_entries_, remote_host_filter_)
{
}
};

60
dbms/src/IO/S3Common.cpp Normal file
View File

@ -0,0 +1,60 @@
#include <IO/S3Common.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <iterator>
#include <sstream>
#include <Poco/Base64Encoder.h>
#include <Poco/HMACEngine.h>
#include <Poco/SHA1Engine.h>
#include <Poco/URI.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_FORMAT_DATETIME;
}
void S3Helper::authenticateRequest(
Poco::Net::HTTPRequest & request,
const String & access_key_id,
const String & secret_access_key)
{
/// See https://docs.aws.amazon.com/AmazonS3/latest/dev/RESTAuthentication.html
if (access_key_id.empty())
return;
/// Limitations:
/// 1. Virtual hosted-style requests are not supported (e.g. `http://johnsmith.net.s3.amazonaws.com/homepage.html`).
/// 2. AMZ headers are not supported (TODO).
if (!request.has("Date"))
{
WriteBufferFromOwnString out;
writeDateTimeTextRFC1123(time(nullptr), out, DateLUT::instance("UTC"));
request.set("Date", out.str());
}
String string_to_sign = request.getMethod() + "\n"
+ request.get("Content-MD5", "") + "\n"
+ request.get("Content-Type", "") + "\n"
+ request.get("Date") + "\n"
+ Poco::URI(request.getURI()).getPathAndQuery();
Poco::HMACEngine<Poco::SHA1Engine> engine(secret_access_key);
engine.update(string_to_sign);
auto digest = engine.digest();
std::ostringstream signature;
Poco::Base64Encoder encoder(signature);
std::copy(digest.begin(), digest.end(), std::ostream_iterator<char>(encoder));
encoder.close();
request.set("Authorization", "AWS " + access_key_id + ":" + signature.str());
}
}

19
dbms/src/IO/S3Common.h Normal file
View File

@ -0,0 +1,19 @@
#pragma once
#include <Core/Types.h>
#include <Poco/Net/HTTPRequest.h>
namespace DB
{
namespace S3Helper
{
void authenticateRequest(
Poco::Net::HTTPRequest & request,
const String & access_key_id,
const String & secret_access_key);
};
}

View File

@ -90,6 +90,9 @@ public:
++pos;
}
virtual void sync() {}
virtual void finalize() {}
private:
/** Write the data in the buffer (from the beginning of the buffer to the current position).
* Throw an exception if something is wrong.

View File

@ -51,7 +51,7 @@ private:
/// Prepare an asynchronous request.
void prepare();
///
void finalize();
void finalize() override;
private:
/// Buffer for asynchronous data writes.

View File

@ -25,7 +25,7 @@ public:
~WriteBufferFromHDFS() override;
void sync();
void sync() override;
};
}
#endif

View File

@ -113,8 +113,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response.set("Content-Encoding", "gzip");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
else if (compression_method == CompressionMethod::Zlib)
@ -125,8 +125,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response.set("Content-Encoding", "deflate");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf;
}
#if USE_BROTLI
@ -138,7 +138,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response.set("Content-Encoding", "br");
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr);
out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin());
out = &*brotli_buf;
}
@ -155,7 +155,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response_body_ostr = &(response.send());
#endif
out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin());
out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr, working_buffer.size(), working_buffer.begin());
out = &*out_raw;
}
}

View File

@ -60,7 +60,7 @@ private:
std::ostream * response_header_ostr = nullptr;
#endif
std::optional<WriteBufferFromOStream> out_raw;
std::unique_ptr<WriteBufferFromOStream> out_raw;
std::optional<ZlibDeflatingWriteBuffer> deflating_buf;
#if USE_BROTLI
std::optional<BrotliWriteBuffer> brotli_buf;
@ -109,7 +109,7 @@ public:
/// Use after the data has possibly been sent and no error happened (and thus you do not plan
/// to change response HTTP code.
/// This method is idempotent.
void finalize();
void finalize() override;
/// Turn compression on or off.
/// The setting has any effect only if HTTP headers haven't been sent yet.

View File

@ -1,5 +1,6 @@
#include <IO/WriteBufferFromS3.h>
#include <IO/S3Common.h>
#include <IO/WriteHelpers.h>
#include <Poco/DOM/AutoPtr.h>
@ -30,22 +31,24 @@ namespace ErrorCodes
WriteBufferFromS3::WriteBufferFromS3(
const Poco::URI & uri_,
const String & access_key_id_,
const String & secret_access_key_,
size_t minimum_upload_part_size_,
const ConnectionTimeouts & timeouts_,
const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_
)
: BufferWithOwnMemory<WriteBuffer>(buffer_size_, nullptr, 0)
const RemoteHostFilter & remote_host_filter_)
: BufferWithOwnMemory<WriteBuffer>(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0)
, uri {uri_}
, access_key_id {access_key_id_}
, secret_access_key {secret_access_key_}
, minimum_upload_part_size {minimum_upload_part_size_}
, timeouts {timeouts_}
, auth_request {Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1}
, temporary_buffer {std::make_unique<WriteBufferFromString>(buffer_string)}
, last_part_size {0}
, remote_host_filter(remote_host_filter_)
{
if (!credentials.getUsername().empty())
credentials.authenticate(auth_request);
initiate();
/// FIXME: Implement rest of S3 authorization.
}
@ -113,11 +116,7 @@ void WriteBufferFromS3::initiate()
request_ptr = std::make_unique<Poco::Net::HTTPRequest>(Poco::Net::HTTPRequest::HTTP_POST, initiate_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
request_ptr->setHost(initiate_uri.getHost()); // use original, not resolved host name in header
if (auth_request.hasCredentials())
{
Poco::Net::HTTPBasicCredentials credentials(auth_request);
credentials.authenticate(*request_ptr);
}
S3Helper::authenticateRequest(*request_ptr, access_key_id, secret_access_key);
request_ptr->setContentLength(0);
@ -137,6 +136,7 @@ void WriteBufferFromS3::initiate()
break;
initiate_uri = location_iterator->second;
remote_host_filter.checkURL(initiate_uri);
}
assertResponseIsOk(*request_ptr, response, *istr);
@ -179,11 +179,7 @@ void WriteBufferFromS3::writePart(const String & data)
request_ptr = std::make_unique<Poco::Net::HTTPRequest>(Poco::Net::HTTPRequest::HTTP_PUT, part_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
request_ptr->setHost(part_uri.getHost()); // use original, not resolved host name in header
if (auth_request.hasCredentials())
{
Poco::Net::HTTPBasicCredentials credentials(auth_request);
credentials.authenticate(*request_ptr);
}
S3Helper::authenticateRequest(*request_ptr, access_key_id, secret_access_key);
request_ptr->setExpectContinue(true);
@ -252,11 +248,7 @@ void WriteBufferFromS3::complete()
request_ptr = std::make_unique<Poco::Net::HTTPRequest>(Poco::Net::HTTPRequest::HTTP_POST, complete_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1);
request_ptr->setHost(complete_uri.getHost()); // use original, not resolved host name in header
if (auth_request.hasCredentials())
{
Poco::Net::HTTPBasicCredentials credentials(auth_request);
credentials.authenticate(*request_ptr);
}
S3Helper::authenticateRequest(*request_ptr, access_key_id, secret_access_key);
request_ptr->setExpectContinue(true);

View File

@ -21,12 +21,14 @@ class WriteBufferFromS3 : public BufferWithOwnMemory<WriteBuffer>
{
private:
Poco::URI uri;
String access_key_id;
String secret_access_key;
size_t minimum_upload_part_size;
ConnectionTimeouts timeouts;
Poco::Net::HTTPRequest auth_request;
String buffer_string;
std::unique_ptr<WriteBufferFromString> temporary_buffer;
size_t last_part_size;
RemoteHostFilter remote_host_filter;
/// Upload in S3 is made in parts.
/// We initiate upload, then upload each part and get ETag as a response, and then finish upload with listing all our parts.
@ -35,15 +37,16 @@ private:
public:
explicit WriteBufferFromS3(const Poco::URI & uri,
const String & access_key_id,
const String & secret_access_key,
size_t minimum_upload_part_size_,
const ConnectionTimeouts & timeouts = {},
const Poco::Net::HTTPBasicCredentials & credentials = {},
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE);
const RemoteHostFilter & remote_host_filter_ = {});
void nextImpl() override;
/// Receives response from the server after sending all data.
void finalize();
void finalize() override;
~WriteBufferFromS3() override;

View File

@ -20,11 +20,13 @@
#include <Common/UInt128.h>
#include <Common/intExp.h>
#include <IO/CompressionMethod.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteIntText.h>
#include <IO/VarInt.h>
#include <IO/DoubleConverter.h>
#include <IO/WriteBufferFromString.h>
#include <IO/ZlibDeflatingWriteBuffer.h>
#include <Formats/FormatSettings.h>
@ -566,11 +568,8 @@ inline void writeUUIDText(const UUID & uuid, WriteBuffer & buf)
buf.write(s, sizeof(s));
}
/// in YYYY-MM-DD format
template <char delimiter = '-'>
inline void writeDateText(const LocalDate & date, WriteBuffer & buf)
{
static const char digits[201] =
static const char digits100[201] =
"00010203040506070809"
"10111213141516171819"
"20212223242526272829"
@ -582,29 +581,33 @@ inline void writeDateText(const LocalDate & date, WriteBuffer & buf)
"80818283848586878889"
"90919293949596979899";
/// in YYYY-MM-DD format
template <char delimiter = '-'>
inline void writeDateText(const LocalDate & date, WriteBuffer & buf)
{
if (buf.position() + 10 <= buf.buffer().end())
{
memcpy(buf.position(), &digits[date.year() / 100 * 2], 2);
memcpy(buf.position(), &digits100[date.year() / 100 * 2], 2);
buf.position() += 2;
memcpy(buf.position(), &digits[date.year() % 100 * 2], 2);
memcpy(buf.position(), &digits100[date.year() % 100 * 2], 2);
buf.position() += 2;
*buf.position() = delimiter;
++buf.position();
memcpy(buf.position(), &digits[date.month() * 2], 2);
memcpy(buf.position(), &digits100[date.month() * 2], 2);
buf.position() += 2;
*buf.position() = delimiter;
++buf.position();
memcpy(buf.position(), &digits[date.day() * 2], 2);
memcpy(buf.position(), &digits100[date.day() * 2], 2);
buf.position() += 2;
}
else
{
buf.write(&digits[date.year() / 100 * 2], 2);
buf.write(&digits[date.year() % 100 * 2], 2);
buf.write(&digits100[date.year() / 100 * 2], 2);
buf.write(&digits100[date.year() % 100 * 2], 2);
buf.write(delimiter);
buf.write(&digits[date.month() * 2], 2);
buf.write(&digits100[date.month() * 2], 2);
buf.write(delimiter);
buf.write(&digits[date.day() * 2], 2);
buf.write(&digits100[date.day() * 2], 2);
}
}
@ -626,59 +629,47 @@ inline void writeDateText(DayNum date, WriteBuffer & buf)
template <char date_delimeter = '-', char time_delimeter = ':', char between_date_time_delimiter = ' '>
inline void writeDateTimeText(const LocalDateTime & datetime, WriteBuffer & buf)
{
static const char digits[201] =
"00010203040506070809"
"10111213141516171819"
"20212223242526272829"
"30313233343536373839"
"40414243444546474849"
"50515253545556575859"
"60616263646566676869"
"70717273747576777879"
"80818283848586878889"
"90919293949596979899";
if (buf.position() + 19 <= buf.buffer().end())
{
memcpy(buf.position(), &digits[datetime.year() / 100 * 2], 2);
memcpy(buf.position(), &digits100[datetime.year() / 100 * 2], 2);
buf.position() += 2;
memcpy(buf.position(), &digits[datetime.year() % 100 * 2], 2);
memcpy(buf.position(), &digits100[datetime.year() % 100 * 2], 2);
buf.position() += 2;
*buf.position() = date_delimeter;
++buf.position();
memcpy(buf.position(), &digits[datetime.month() * 2], 2);
memcpy(buf.position(), &digits100[datetime.month() * 2], 2);
buf.position() += 2;
*buf.position() = date_delimeter;
++buf.position();
memcpy(buf.position(), &digits[datetime.day() * 2], 2);
memcpy(buf.position(), &digits100[datetime.day() * 2], 2);
buf.position() += 2;
*buf.position() = between_date_time_delimiter;
++buf.position();
memcpy(buf.position(), &digits[datetime.hour() * 2], 2);
memcpy(buf.position(), &digits100[datetime.hour() * 2], 2);
buf.position() += 2;
*buf.position() = time_delimeter;
++buf.position();
memcpy(buf.position(), &digits[datetime.minute() * 2], 2);
memcpy(buf.position(), &digits100[datetime.minute() * 2], 2);
buf.position() += 2;
*buf.position() = time_delimeter;
++buf.position();
memcpy(buf.position(), &digits[datetime.second() * 2], 2);
memcpy(buf.position(), &digits100[datetime.second() * 2], 2);
buf.position() += 2;
}
else
{
buf.write(&digits[datetime.year() / 100 * 2], 2);
buf.write(&digits[datetime.year() % 100 * 2], 2);
buf.write(&digits100[datetime.year() / 100 * 2], 2);
buf.write(&digits100[datetime.year() % 100 * 2], 2);
buf.write(date_delimeter);
buf.write(&digits[datetime.month() * 2], 2);
buf.write(&digits100[datetime.month() * 2], 2);
buf.write(date_delimeter);
buf.write(&digits[datetime.day() * 2], 2);
buf.write(&digits100[datetime.day() * 2], 2);
buf.write(between_date_time_delimiter);
buf.write(&digits[datetime.hour() * 2], 2);
buf.write(&digits100[datetime.hour() * 2], 2);
buf.write(time_delimeter);
buf.write(&digits[datetime.minute() * 2], 2);
buf.write(&digits100[datetime.minute() * 2], 2);
buf.write(time_delimeter);
buf.write(&digits[datetime.second() * 2], 2);
buf.write(&digits100[datetime.second() * 2], 2);
}
}
@ -705,6 +696,33 @@ inline void writeDateTimeText(time_t datetime, WriteBuffer & buf, const DateLUTI
}
/// In the RFC 1123 format: "Tue, 03 Dec 2019 00:11:50 GMT". You must provide GMT DateLUT.
/// This is needed for HTTP requests.
inline void writeDateTimeTextRFC1123(time_t datetime, WriteBuffer & buf, const DateLUTImpl & date_lut)
{
const auto & values = date_lut.getValues(datetime);
static const char week_days[3 * 8 + 1] = "XXX" "Mon" "Tue" "Wed" "Thu" "Fri" "Sat" "Sun";
static const char months[3 * 13 + 1] = "XXX" "Jan" "Feb" "Mar" "Apr" "May" "Jun" "Jul" "Aug" "Sep" "Oct" "Nov" "Dec";
buf.write(&week_days[values.day_of_week * 3], 3);
buf.write(", ", 2);
buf.write(&digits100[values.day_of_month * 2], 2);
buf.write(' ');
buf.write(&months[values.month * 3], 3);
buf.write(' ');
buf.write(&digits100[values.year / 100 * 2], 2);
buf.write(&digits100[values.year % 100 * 2], 2);
buf.write(' ');
buf.write(&digits100[date_lut.toHour(datetime) * 2], 2);
buf.write(':');
buf.write(&digits100[date_lut.toMinute(datetime) * 2], 2);
buf.write(':');
buf.write(&digits100[date_lut.toSecond(datetime) * 2], 2);
buf.write(" GMT", 4);
}
/// Methods for output in binary format.
template <typename T>
inline std::enable_if_t<is_arithmetic_v<T>, void>
@ -905,4 +923,16 @@ inline String toString(const T & x)
writeText(x, buf);
return buf.str();
}
template <class TWriteBuffer, class... Types>
std::unique_ptr<WriteBuffer> getWriteBuffer(const DB::CompressionMethod method, Types&&... args)
{
if (method == DB::CompressionMethod::Gzip)
{
auto write_buf = std::make_unique<TWriteBuffer>(std::forward<Types>(args)...);
return std::make_unique<ZlibDeflatingWriteBuffer>(std::move(write_buf), method, 1 /* compression level */);
}
return std::make_unique<TWriteBuffer>(args...);
}
}

View File

@ -6,14 +6,14 @@ namespace DB
{
ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
WriteBuffer & out_,
std::unique_ptr<WriteBuffer> out_,
CompressionMethod compression_method,
int compression_level,
size_t buf_size,
char * existing_memory,
size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
, out(out_)
, out(std::move(out_))
{
zstr.zalloc = nullptr;
zstr.zfree = nullptr;
@ -64,18 +64,18 @@ void ZlibDeflatingWriteBuffer::nextImpl()
do
{
out.nextIfAtEnd();
zstr.next_out = reinterpret_cast<unsigned char *>(out.position());
zstr.avail_out = out.buffer().end() - out.position();
out->nextIfAtEnd();
zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
zstr.avail_out = out->buffer().end() - out->position();
int rc = deflate(&zstr, Z_NO_FLUSH);
out.position() = out.buffer().end() - zstr.avail_out;
out->position() = out->buffer().end() - zstr.avail_out;
// Unpoison the result of deflate explicitly. It uses some custom SSE algo
// for computing CRC32, and it looks like msan is unable to comprehend
// it fully, so it complains about the resulting value depending on the
// uninitialized padding of the input buffer.
__msan_unpoison(out.position(), zstr.avail_out);
__msan_unpoison(out->position(), zstr.avail_out);
if (rc != Z_OK)
throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED);
@ -92,18 +92,18 @@ void ZlibDeflatingWriteBuffer::finish()
while (true)
{
out.nextIfAtEnd();
zstr.next_out = reinterpret_cast<unsigned char *>(out.position());
zstr.avail_out = out.buffer().end() - out.position();
out->nextIfAtEnd();
zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
zstr.avail_out = out->buffer().end() - out->position();
int rc = deflate(&zstr, Z_FINISH);
out.position() = out.buffer().end() - zstr.avail_out;
out->position() = out->buffer().end() - zstr.avail_out;
// Unpoison the result of deflate explicitly. It uses some custom SSE algo
// for computing CRC32, and it looks like msan is unable to comprehend
// it fully, so it complains about the resulting value depending on the
// uninitialized padding of the input buffer.
__msan_unpoison(out.position(), zstr.avail_out);
__msan_unpoison(out->position(), zstr.avail_out);
if (rc == Z_STREAM_END)
{

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