Merge branch 'master' into yandex-to-clickhouse-in-configs

This commit is contained in:
alexey-milovidov 2021-09-23 20:49:40 +03:00 committed by GitHub
commit b8083929a6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
134 changed files with 3439 additions and 1177 deletions

View File

@ -100,8 +100,8 @@ For a description of parameters, see the [CREATE query description](../../../sql
- `min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes.
<a name="mergetree_setting-merge_with_ttl_timeout"></a>
- `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with delete TTL. Default value: `14400` seconds (4 hours).
- `merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours).
- `try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours).
- `merge_with_recompression_ttl_timeout` — Minimum delay in seconds before repeating a merge with recompression TTL. Default value: `14400` seconds (4 hours).
- `try_fetch_recompressed_part_timeout` — Timeout (in seconds) before starting merge with recompression. During this time ClickHouse tries to fetch recompressed part from replica which assigned this merge with recompression. Default value: `7200` seconds (2 hours).
- `write_final_mark` — Enables or disables writing the final index mark at the end of data part (after the last byte). Default value: 1. Dont turn it off.
- `merge_max_block_size` — Maximum number of rows in block for merge operations. Default value: 8192.
- `storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes).
@ -335,7 +335,16 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234
The optional `false_positive` parameter is the probability of receiving a false positive response from the filter. Possible values: (0, 1). Default value: 0.025.
Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`.
Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`, `Map`.
For `Map` data type client can specify if index should be created for keys or values using [mapKeys](../../../sql-reference/functions/tuple-map-functions.md#mapkeys) or [mapValues](../../../sql-reference/functions/tuple-map-functions.md#mapvalues) function.
Example of index creation for `Map` data type
```
INDEX map_key_index mapKeys(map_column) TYPE bloom_filter GRANULARITY 1
INDEX map_key_index mapValues(map_column) TYPE bloom_filter GRANULARITY 1
```
The following functions can use it: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md).
@ -398,7 +407,7 @@ Projections are an experimental feature. To enable them you must set the [allow_
Projections are not supported in the `SELECT` statements with the [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier.
### Projection Query {#projection-query}
A projection query is what defines a projection. It implicitly selects data from the parent table.
A projection query is what defines a projection. It implicitly selects data from the parent table.
**Syntax**
```sql
@ -548,7 +557,7 @@ ORDER BY d
TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1;
```
Creating a table, where expired rows are recompressed:
Creating a table, where expired rows are recompressed:
```sql
CREATE TABLE table_for_recompression

View File

@ -7,5 +7,5 @@ toc_title: GitHub Events
Dataset contains all events on GitHub from 2011 to Dec 6 2020, the size is 3.1 billion records. Download size is 75 GB and it will require up to 200 GB space on disk if stored in a table with lz4 compression.
Full dataset description, insights, download instruction and interactive queries are posted [here](https://github-sql.github.io/explorer/).
Full dataset description, insights, download instruction and interactive queries are posted [here](https://ghe.clickhouse.tech/).

View File

@ -1553,18 +1553,20 @@ ClickHouse supports reading and writing [MessagePack](https://msgpack.org/) data
### Data Types Matching {#data-types-matching-msgpack}
| MsgPack data type | ClickHouse data type |
|---------------------------------|----------------------------------------------------------------------------------|
| `uint N`, `positive fixint` | [UIntN](../sql-reference/data-types/int-uint.md) |
| `int N` | [IntN](../sql-reference/data-types/int-uint.md) |
| `fixstr`, `str 8`, `str 16`, `str 32` | [String](../sql-reference/data-types/string.md), [FixedString](../sql-reference/data-types/fixedstring.md) |
| `float 32` | [Float32](../sql-reference/data-types/float.md) |
| `float 64` | [Float64](../sql-reference/data-types/float.md) |
| `uint 16` | [Date](../sql-reference/data-types/date.md) |
| `uint 32` | [DateTime](../sql-reference/data-types/datetime.md) |
| `uint 64` | [DateTime64](../sql-reference/data-types/datetime.md) |
| `fixarray`, `array 16`, `array 32`| [Array](../sql-reference/data-types/array.md) |
| `nil` | [Nothing](../sql-reference/data-types/special-data-types/nothing.md) |
| MessagePack data type (`INSERT`) | ClickHouse data type | MessagePack data type (`SELECT`) |
|--------------------------------------------------------------------|-----------------------------------------------------------|------------------------------------|
| `uint N`, `positive fixint` | [UIntN](../sql-reference/data-types/int-uint.md) | `uint N` |
| `int N` | [IntN](../sql-reference/data-types/int-uint.md) | `int N` |
| `bool` | [UInt8](../sql-reference/data-types/int-uint.md) | `uint 8` |
| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [String](../sql-reference/data-types/string.md) | `bin 8`, `bin 16`, `bin 32` |
| `fixstr`, `str 8`, `str 16`, `str 32`, `bin 8`, `bin 16`, `bin 32` | [FixedString](../sql-reference/data-types/fixedstring.md) | `bin 8`, `bin 16`, `bin 32` |
| `float 32` | [Float32](../sql-reference/data-types/float.md) | `float 32` |
| `float 64` | [Float64](../sql-reference/data-types/float.md) | `float 64` |
| `uint 16` | [Date](../sql-reference/data-types/date.md) | `uint 16` |
| `uint 32` | [DateTime](../sql-reference/data-types/datetime.md) | `uint 32` |
| `uint 64` | [DateTime64](../sql-reference/data-types/datetime.md) | `uint 64` |
| `fixarray`, `array 16`, `array 32` | [Array](../sql-reference/data-types/array.md) | `fixarray`, `array 16`, `array 32` |
| `fixmap`, `map 16`, `map 32` | [Map](../sql-reference/data-types/map.md) | `fixmap`, `map 16`, `map 32` |
Example:

View File

@ -810,7 +810,7 @@ If ClickHouse should read more than `merge_tree_max_bytes_to_use_cache` bytes in
The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks.
Possible value:
Possible values:
- Any positive integer.
@ -818,23 +818,23 @@ Default value: 2013265920.
## merge_tree_clear_old_temporary_directories_interval_seconds {#setting-merge-tree-clear-old-temporary-directories-interval-seconds}
The interval in seconds for ClickHouse to execute the cleanup old temporary directories.
Sets the interval in seconds for ClickHouse to execute the cleanup of old temporary directories.
Possible value:
Possible values:
- Any positive integer.
Default value: 60.
Default value: `60` seconds.
## merge_tree_clear_old_parts_interval_seconds {#setting-merge-tree-clear-old-parts-interval-seconds}
The interval in seconds for ClickHouse to execute the cleanup old parts, WALs, and mutations.
Sets the interval in seconds for ClickHouse to execute the cleanup of old parts, WALs, and mutations.
Possible value:
Possible values:
- Any positive integer.
Default value: 1.
Default value: `1` second.
## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io}

View File

@ -66,6 +66,8 @@ ClickHouse-specific aggregate functions:
- [quantileDeterministic](../../../sql-reference/aggregate-functions/reference/quantiledeterministic.md)
- [quantileTDigest](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md)
- [quantileTDigestWeighted](../../../sql-reference/aggregate-functions/reference/quantiletdigestweighted.md)
- [quantileBFloat16](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16)
- [quantileBFloat16Weighted](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16weighted)
- [simpleLinearRegression](../../../sql-reference/aggregate-functions/reference/simplelinearregression.md)
- [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md)
- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md)

View File

@ -58,6 +58,10 @@ Result:
```
Note that all floating point values in the example are truncated to 1.0 when converting to `bfloat16`.
# quantileBFloat16Weighted {#quantilebfloat16weighted}
Like `quantileBFloat16` but takes into account the weight of each sequence member.
**See Also**
- [median](../../../sql-reference/aggregate-functions/reference/median.md#median)

View File

@ -107,7 +107,7 @@ bitmapSubsetLimit(bitmap, range_start, cardinality_limit)
The subset.
Type: `Bitmap object`.
Type: [Bitmap object](#bitmap_functions-bitmapbuild).
**Example**
@ -125,9 +125,9 @@ Result:
└───────────────────────────┘
```
## subBitmap {#subBitmap}
## subBitmap {#subbitmap}
Creates a subset of bitmap limit the results to `cardinality_limit` with offset of `offset`.
Returns the bitmap elements, starting from the `offset` position. The number of returned elements is limited by the `cardinality_limit` parameter. Analog of the [substring](string-functions.md#substring)) string function, but for bitmap.
**Syntax**
@ -137,15 +137,15 @@ subBitmap(bitmap, offset, cardinality_limit)
**Arguments**
- `bitmap` [Bitmap object](#bitmap_functions-bitmapbuild).
- `offset` the number of offsets. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `cardinality_limit` The subset cardinality upper limit. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `bitmap` The bitmap. Type: [Bitmap object](#bitmap_functions-bitmapbuild).
- `offset` The position of the first element of the subset. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
- `cardinality_limit` The maximum number of elements in the subset. Type: [UInt32](../../sql-reference/data-types/int-uint.md).
**Returned value**
The subset.
Type: `Bitmap object`.
Type: [Bitmap object](#bitmap_functions-bitmapbuild).
**Example**

View File

@ -11,7 +11,7 @@ ALTER TABLE [db].name [ON CLUSTER cluster] MODIFY ORDER BY new_expression
The command changes the [sorting key](../../../engines/table-engines/mergetree-family/mergetree.md) of the table to `new_expression` (an expression or a tuple of expressions). Primary key remains the same.
The command is lightweight in a sense that it only changes metadata. To keep the property that data part rows are ordered by the sorting key expression you cannot add expressions containing existing columns to the sorting key (only columns added by the `ADD COLUMN` command in the same `ALTER` query).
The command is lightweight in a sense that it only changes metadata. To keep the property that data part rows are ordered by the sorting key expression you cannot add expressions containing existing columns to the sorting key (only columns added by the `ADD COLUMN` command in the same `ALTER` query, without default column value).
!!! note "Note"
It only works for tables in the [`MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) family (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) tables).

View File

@ -801,12 +801,32 @@ ClickHouse может парсить только базовый формат `Y
Кэш несжатых блоков хранит данные, извлечённые при выполнении запросов. ClickHouse использует кэш для ускорения ответов на повторяющиеся небольшие запросы. Настройка защищает кэш от переполнения. Настройка сервера [uncompressed_cache_size](../server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) определяет размер кэша несжатых блоков.
Возможное значение:
Возможные значения:
- Положительное целое число.
Значение по умолчанию: 2013265920.
## merge_tree_clear_old_temporary_directories_interval_seconds {#setting-merge-tree-clear-old-temporary-directories-interval-seconds}
Задает интервал в секундах для удаления старых временных каталогов на сервере ClickHouse.
Возможные значения:
- Положительное целое число.
Значение по умолчанию: `60` секунд.
## merge_tree_clear_old_parts_interval_seconds {#setting-merge-tree-clear-old-parts-interval-seconds}
Задает интервал в секундах для удаления старых кусков данных, журналов предзаписи (WAL) и мутаций на сервере ClickHouse .
Возможные значения:
- Положительное целое число.
Значение по умолчанию: `1` секунда.
## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io}
Минимальный объём данных, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск.

View File

@ -61,6 +61,8 @@ toc_hidden: true
- [quantileDeterministic](../../../sql-reference/aggregate-functions/reference/quantiledeterministic.md)
- [quantileTDigest](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md)
- [quantileTDigestWeighted](../../../sql-reference/aggregate-functions/reference/quantiletdigestweighted.md)
- [quantileBFloat16](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16)
- [quantileBFloat16Weighted](../../../sql-reference/aggregate-functions/reference/quantilebfloat16.md#quantilebfloat16weighted)
- [simpleLinearRegression](../../../sql-reference/aggregate-functions/reference/simplelinearregression.md)
- [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md)
- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md)

View File

@ -58,6 +58,10 @@ SELECT quantileBFloat16(0.75)(a), quantileBFloat16(0.75)(b) FROM example_table;
```
Обратите внимание, что все числа с плавающей точкой в примере были округлены до 1.0 при преобразовании к `bfloat16`.
# quantileBFloat16Weighted {#quantilebfloat16weighted}
Версия функции `quantileBFloat16`, которая учитывает вес каждого элемента последовательности.
**См. также**
- [median](../../../sql-reference/aggregate-functions/reference/median.md#median)

View File

@ -66,15 +66,14 @@ bitmapSubsetLimit(bitmap, range_start, cardinality_limit)
**Аргументы**
- `bitmap` битмап. [Bitmap object](#bitmap_functions-bitmapbuild).
- `range_start` начальная точка подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md#bitmap-functions).
- `cardinality_limit` Верхний предел подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md#bitmap-functions).
- `cardinality_limit` верхний предел подмножества. [UInt32](../../sql-reference/functions/bitmap-functions.md#bitmap-functions).
**Возвращаемое значение**
Подмножество битмапа.
Тип: `Bitmap object`.
Тип: [Bitmap object](#bitmap_functions-bitmapbuild).
**Пример**
@ -92,6 +91,44 @@ SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12
└───────────────────────────┘
```
## subBitmap {#subbitmap}
Возвращает элементы битмапа, начиная с позиции `offset`. Число возвращаемых элементов ограничивается параметром `cardinality_limit`. Аналог строковой функции [substring](string-functions.md#substring)), но для битмапа.
**Синтаксис**
``` sql
subBitmap(bitmap, offset, cardinality_limit)
```
**Аргументы**
- `bitmap` битмап. Тип: [Bitmap object](#bitmap_functions-bitmapbuild).
- `offset` позиция первого элемента возвращаемого подмножества. Тип: [UInt32](../../sql-reference/data-types/int-uint.md).
- `cardinality_limit` максимальное число элементов возвращаемого подмножества. Тип: [UInt32](../../sql-reference/data-types/int-uint.md).
**Возвращаемое значение**
Подмножество битмапа.
Тип: [Bitmap object](#bitmap_functions-bitmapbuild).
**Пример**
Запрос:
``` sql
SELECT bitmapToArray(subBitmap(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(10), toUInt32(10))) AS res;
```
Результат:
``` text
┌─res─────────────────────────────┐
│ [10,11,12,13,14,15,16,17,18,19] │
└─────────────────────────────────┘
```
## bitmapContains {#bitmap_functions-bitmapcontains}
Проверяет вхождение элемента в битовый массив.

View File

@ -93,7 +93,7 @@ def build_for_lang(lang, args):
site_url=f'{website_url}/docs/{lang}/',
docs_dir=os.path.join(args.docs_dir, lang),
site_dir=site_dir,
strict=False, # TODO: fix issues and return True
strict=True,
theme=theme_cfg,
copyright='©20162021 ClickHouse, Inc.',
use_directory_urls=True,

View File

@ -419,7 +419,12 @@ void Connection::sendQuery(
if (!connected)
connect(timeouts);
TimeoutSetter timeout_setter(*socket, timeouts.send_timeout, timeouts.receive_timeout, true);
/// Query is not executed within sendQuery() function.
///
/// And what this means that temporary timeout (via TimeoutSetter) is not
/// enough, since next query can use timeout from the previous query in this case.
socket->setReceiveTimeout(timeouts.receive_timeout);
socket->setSendTimeout(timeouts.send_timeout);
if (settings)
{

View File

@ -534,6 +534,13 @@ ExecutionStatus ExecutionStatus::fromCurrentException(const std::string & start_
return ExecutionStatus(getCurrentExceptionCode(), msg);
}
ExecutionStatus ExecutionStatus::fromText(const std::string & data)
{
ExecutionStatus status;
status.deserializeText(data);
return status;
}
ParsingException::ParsingException() = default;
ParsingException::ParsingException(const std::string & msg, int code)
: Exception(msg, code)

View File

@ -184,6 +184,8 @@ struct ExecutionStatus
static ExecutionStatus fromCurrentException(const std::string & start_of_message = "");
static ExecutionStatus fromText(const std::string & data);
std::string serializeText() const;
void deserializeText(const std::string & data);

View File

@ -1,6 +1,5 @@
#include <Common/SettingsChanges.h>
namespace DB
{
namespace

View File

@ -5,6 +5,9 @@
namespace DB
{
class IColumn;
struct SettingChange
{
String name;

View File

@ -5,6 +5,8 @@
#include "Utils.h"
#include <Common/parseRemoteDescription.h>
#include <Common/Exception.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
namespace DB
{
@ -18,7 +20,7 @@ namespace postgres
{
PoolWithFailover::PoolWithFailover(
const Poco::Util::AbstractConfiguration & config, const String & config_prefix,
const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority,
size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_)
: pool_wait_timeout(pool_wait_timeout_)
, max_tries(max_tries_)
@ -26,45 +28,19 @@ PoolWithFailover::PoolWithFailover(
LOG_TRACE(&Poco::Logger::get("PostgreSQLConnectionPool"), "PostgreSQL connection pool size: {}, connection wait timeout: {}, max failover tries: {}",
pool_size, pool_wait_timeout, max_tries_);
auto db = config.getString(config_prefix + ".db", "");
auto host = config.getString(config_prefix + ".host", "");
auto port = config.getUInt(config_prefix + ".port", 0);
auto user = config.getString(config_prefix + ".user", "");
auto password = config.getString(config_prefix + ".password", "");
if (config.has(config_prefix + ".replica"))
for (const auto & [priority, configurations] : configurations_by_priority)
{
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_prefix, config_keys);
for (const auto & config_key : config_keys)
for (const auto & replica_configuration : configurations)
{
if (config_key.starts_with("replica"))
{
std::string replica_name = config_prefix + "." + config_key;
size_t priority = config.getInt(replica_name + ".priority", 0);
auto replica_host = config.getString(replica_name + ".host", host);
auto replica_port = config.getUInt(replica_name + ".port", port);
auto replica_user = config.getString(replica_name + ".user", user);
auto replica_password = config.getString(replica_name + ".password", password);
auto connection_string = formatConnectionString(db, replica_host, replica_port, replica_user, replica_password).first;
replicas_with_priority[priority].emplace_back(connection_string, pool_size);
}
auto connection_string = formatConnectionString(replica_configuration.database,
replica_configuration.host, replica_configuration.port, replica_configuration.username, replica_configuration.password).first;
replicas_with_priority[priority].emplace_back(connection_string, pool_size, getConnectionForLog(replica_configuration.host, replica_configuration.port));
}
}
else
{
auto connection_string = formatConnectionString(db, host, port, user, password).first;
replicas_with_priority[0].emplace_back(connection_string, pool_size);
}
}
PoolWithFailover::PoolWithFailover(
const std::string & database,
const RemoteDescription & addresses,
const std::string & user, const std::string & password,
const DB::StoragePostgreSQLConfiguration & configuration,
size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_)
: pool_wait_timeout(pool_wait_timeout_)
, max_tries(max_tries_)
@ -73,11 +49,11 @@ PoolWithFailover::PoolWithFailover(
pool_size, pool_wait_timeout, max_tries_);
/// Replicas have the same priority, but traversed replicas are moved to the end of the queue.
for (const auto & [host, port] : addresses)
for (const auto & [host, port] : configuration.addresses)
{
LOG_DEBUG(&Poco::Logger::get("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port);
auto connection_string = formatConnectionString(database, host, port, user, password).first;
replicas_with_priority[0].emplace_back(connection_string, pool_size);
auto connection_string = formatConnectionString(configuration.database, host, port, configuration.username, configuration.password).first;
replicas_with_priority[0].emplace_back(connection_string, pool_size, getConnectionForLog(host, port));
}
}
@ -85,6 +61,7 @@ ConnectionHolderPtr PoolWithFailover::get()
{
std::lock_guard lock(mutex);
DB::WriteBufferFromOwnString error_message;
for (size_t try_idx = 0; try_idx < max_tries; ++try_idx)
{
for (auto & priority : replicas_with_priority)
@ -115,6 +92,7 @@ ConnectionHolderPtr PoolWithFailover::get()
catch (const pqxx::broken_connection & pqxx_error)
{
LOG_ERROR(log, "Connection error: {}", pqxx_error.what());
error_message << "Try " << try_idx + 1 << ". Connection to `" << replica.name_for_log << "` failed: " << pqxx_error.what() << "\n";
replica.pool->returnObject(std::move(connection));
continue;
@ -136,7 +114,7 @@ ConnectionHolderPtr PoolWithFailover::get()
}
}
throw DB::Exception(DB::ErrorCodes::POSTGRESQL_CONNECTION_FAILURE, "Unable to connect to any of the replicas");
throw DB::Exception(DB::ErrorCodes::POSTGRESQL_CONNECTION_FAILURE, error_message.str());
}
}

View File

@ -11,6 +11,7 @@
#include <mutex>
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace postgres
@ -27,17 +28,13 @@ public:
static constexpr inline auto POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES = 5;
PoolWithFailover(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const DB::ExternalDataSourcesConfigurationByPriority & configurations_by_priority,
size_t pool_size = POSTGRESQL_POOL_DEFAULT_SIZE,
size_t pool_wait_timeout = POSTGRESQL_POOL_WAIT_TIMEOUT,
size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
PoolWithFailover(
const std::string & database,
const RemoteDescription & addresses,
const std::string & user,
const std::string & password,
const DB::StoragePostgreSQLConfiguration & configuration,
size_t pool_size = POSTGRESQL_POOL_DEFAULT_SIZE,
size_t pool_wait_timeout = POSTGRESQL_POOL_WAIT_TIMEOUT,
size_t max_tries_ = POSTGRESQL_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES);
@ -51,9 +48,10 @@ private:
{
String connection_string;
PoolPtr pool;
String name_for_log;
PoolHolder(const String & connection_string_, size_t pool_size)
: connection_string(connection_string_), pool(std::make_shared<Pool>(pool_size)) {}
PoolHolder(const String & connection_string_, size_t pool_size, const String & name_for_log_)
: connection_string(connection_string_), pool(std::make_shared<Pool>(pool_size)), name_for_log(name_for_log_) {}
};
/// Highest priority is 0, the bigger the number in map, the less the priority

View File

@ -3,6 +3,7 @@
#if USE_LIBPQXX
#include <IO/Operators.h>
#include <IO/WriteHelpers.h>
namespace postgres
{
@ -19,6 +20,11 @@ ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, S
return std::make_pair(out.str(), host + ':' + DB::toString(port));
}
String getConnectionForLog(const String & host, UInt16 port)
{
return host + ":" + DB::toString(port);
}
String formatNameForLogs(const String & postgres_database_name, const String & postgres_table_name)
{
/// Logger for StorageMaterializedPostgreSQL - both db and table names.

View File

@ -22,6 +22,8 @@ namespace postgres
ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password);
String getConnectionForLog(const String & host, UInt16 port);
String formatNameForLogs(const String & postgres_database_name, const String & postgres_table_name);
}

View File

@ -386,6 +386,7 @@ class IColumn;
M(Bool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \
M(Bool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \
M(Bool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql', 'postgresql' and 'odbc' table functions.", 0) \
M(Bool, external_table_strict_query, false, "If it is set to true, transforming expression to local filter is forbidden for queries to external tables.", 0) \
\
M(Bool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \
M(UInt64, max_hyperscan_regexp_length, 0, "Max length of regexp than can be used in hyperscan multi-match functions. Zero means unlimited.", 0) \

View File

@ -13,6 +13,7 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/formatAST.h>
#include <Common/Macros.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <filesystem>
#if !defined(ARCADIA_BUILD)
@ -38,6 +39,7 @@
#include <Databases/PostgreSQL/DatabasePostgreSQL.h> // Y_IGNORE
#include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
#include <Storages/StoragePostgreSQL.h>
#endif
#if USE_SQLITE
@ -141,40 +143,66 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
else if (engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "MaterializedMySQL")
{
const ASTFunction * engine = engine_define->engine;
if (!engine->arguments || engine->arguments->children.size() != 4)
throw Exception(
engine_name + " Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.",
ErrorCodes::BAD_ARGUMENTS);
if (!engine->arguments)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name);
StorageMySQLConfiguration configuration;
ASTs & arguments = engine->arguments->children;
arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context);
const auto & host_port = safeGetLiteralValue<String>(arguments[0], engine_name);
const auto & mysql_database_name = safeGetLiteralValue<String>(arguments[1], engine_name);
const auto & mysql_user_name = safeGetLiteralValue<String>(arguments[2], engine_name);
const auto & mysql_user_password = safeGetLiteralValue<String>(arguments[3], engine_name);
if (auto named_collection = getExternalDataSourceConfiguration(arguments, context, true))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
if (!storage_specific_args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.");
}
else
{
if (arguments.size() != 4)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.");
arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context);
const auto & host_port = safeGetLiteralValue<String>(arguments[0], engine_name);
if (engine_name == "MySQL")
{
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
}
else
{
const auto & [remote_host, remote_port] = parseAddress(host_port, 3306);
configuration.host = remote_host;
configuration.port = remote_port;
}
configuration.database = safeGetLiteralValue<String>(arguments[1], engine_name);
configuration.username = safeGetLiteralValue<String>(arguments[2], engine_name);
configuration.password = safeGetLiteralValue<String>(arguments[3], engine_name);
}
try
{
if (engine_name == "MySQL")
{
auto mysql_database_settings = std::make_unique<ConnectionMySQLSettings>();
/// Split into replicas if needed.
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
auto mysql_pool = mysqlxx::PoolWithFailover(mysql_database_name, addresses, mysql_user_name, mysql_user_password);
auto mysql_pool = mysqlxx::PoolWithFailover(configuration.database, configuration.addresses, configuration.username, configuration.password);
mysql_database_settings->loadFromQueryContext(context);
mysql_database_settings->loadFromQuery(*engine_define); /// higher priority
return std::make_shared<DatabaseMySQL>(
context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_database_settings), std::move(mysql_pool));
context, database_name, metadata_path, engine_define, configuration.database, std::move(mysql_database_settings), std::move(mysql_pool));
}
const auto & [remote_host_name, remote_port] = parseAddress(host_port, 3306);
MySQLClient client(remote_host_name, remote_port, mysql_user_name, mysql_user_password);
auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port);
MySQLClient client(configuration.host, configuration.port, configuration.username, configuration.password);
auto mysql_pool = mysqlxx::Pool(configuration.database, configuration.host, configuration.username, configuration.password, configuration.port);
auto materialize_mode_settings = std::make_unique<MaterializedMySQLSettings>();
@ -183,12 +211,12 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
if (create.uuid == UUIDHelpers::Nil)
return std::make_shared<DatabaseMaterializedMySQL<DatabaseOrdinary>>(
context, database_name, metadata_path, uuid, mysql_database_name,
std::move(mysql_pool), std::move(client), std::move(materialize_mode_settings));
context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool),
std::move(client), std::move(materialize_mode_settings));
else
return std::make_shared<DatabaseMaterializedMySQL<DatabaseAtomic>>(
context, database_name, metadata_path, uuid, mysql_database_name,
std::move(mysql_pool), std::move(client), std::move(materialize_mode_settings));
context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool),
std::move(client), std::move(materialize_mode_settings));
}
catch (...)
{
@ -242,77 +270,109 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
else if (engine_name == "PostgreSQL")
{
const ASTFunction * engine = engine_define->engine;
if (!engine->arguments || engine->arguments->children.size() < 4 || engine->arguments->children.size() > 6)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{} Database require `host:port`, `database_name`, `username`, `password` [, `schema` = "", `use_table_cache` = 0].",
engine_name);
if (!engine->arguments)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name);
ASTs & engine_args = engine->arguments->children;
auto use_table_cache = false;
StoragePostgreSQLConfiguration configuration;
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context, true))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
const auto & host_port = safeGetLiteralValue<String>(engine_args[0], engine_name);
const auto & postgres_database_name = safeGetLiteralValue<String>(engine_args[1], engine_name);
const auto & username = safeGetLiteralValue<String>(engine_args[2], engine_name);
const auto & password = safeGetLiteralValue<String>(engine_args[3], engine_name);
configuration.set(common_configuration);
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
String schema;
if (engine->arguments->children.size() >= 5)
schema = safeGetLiteralValue<String>(engine_args[4], engine_name);
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
if (arg_name == "use_table_cache")
use_table_cache = true;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unexpected key-value argument."
"Got: {}, but expected one of:"
"host, port, username, password, database, schema, use_table_cache.", arg_name);
}
}
else
{
if (engine_args.size() < 4 || engine_args.size() > 6)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"PostgreSQL Database require `host:port`, `database_name`, `username`, `password`"
"[, `schema` = "", `use_table_cache` = 0");
auto use_table_cache = 0;
if (engine->arguments->children.size() >= 6)
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
const auto & host_port = safeGetLiteralValue<String>(engine_args[0], engine_name);
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
configuration.database = safeGetLiteralValue<String>(engine_args[1], engine_name);
configuration.username = safeGetLiteralValue<String>(engine_args[2], engine_name);
configuration.password = safeGetLiteralValue<String>(engine_args[3], engine_name);
if (engine_args.size() >= 5)
configuration.schema = safeGetLiteralValue<String>(engine_args[4], engine_name);
}
if (engine_args.size() >= 6)
use_table_cache = safeGetLiteralValue<UInt8>(engine_args[5], engine_name);
/// Split into replicas if needed.
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
/// no connection is made here
auto connection_pool = std::make_shared<postgres::PoolWithFailover>(
postgres_database_name,
addresses,
username, password,
auto pool = std::make_shared<postgres::PoolWithFailover>(configuration,
context->getSettingsRef().postgresql_connection_pool_size,
context->getSettingsRef().postgresql_connection_pool_wait_timeout);
return std::make_shared<DatabasePostgreSQL>(
context, metadata_path, engine_define, database_name, postgres_database_name, schema, connection_pool, use_table_cache);
context, metadata_path, engine_define, database_name, configuration, pool, use_table_cache);
}
else if (engine_name == "MaterializedPostgreSQL")
{
const ASTFunction * engine = engine_define->engine;
if (!engine->arguments || engine->arguments->children.size() != 4)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"{} Database require `host:port`, `database_name`, `username`, `password`.",
engine_name);
}
if (!engine->arguments)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name);
ASTs & engine_args = engine->arguments->children;
StoragePostgreSQLConfiguration configuration;
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context, true))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
const auto & host_port = safeGetLiteralValue<String>(engine_args[0], engine_name);
const auto & postgres_database_name = safeGetLiteralValue<String>(engine_args[1], engine_name);
const auto & username = safeGetLiteralValue<String>(engine_args[2], engine_name);
const auto & password = safeGetLiteralValue<String>(engine_args[3], engine_name);
if (!storage_specific_args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"MaterializedPostgreSQL Database requires only `host`, `port`, `database_name`, `username`, `password`.");
}
else
{
if (engine_args.size() != 4)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"MaterializedPostgreSQL Database require `host:port`, `database_name`, `username`, `password`.");
auto parsed_host_port = parseAddress(host_port, 5432);
auto connection_info = postgres::formatConnectionString(postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
auto parsed_host_port = parseAddress(safeGetLiteralValue<String>(engine_args[0], engine_name), 5432);
configuration.host = parsed_host_port.first;
configuration.port = parsed_host_port.second;
configuration.database = safeGetLiteralValue<String>(engine_args[1], engine_name);
configuration.username = safeGetLiteralValue<String>(engine_args[2], engine_name);
configuration.password = safeGetLiteralValue<String>(engine_args[3], engine_name);
}
auto connection_info = postgres::formatConnectionString(
configuration.database, configuration.host, configuration.port, configuration.username, configuration.password);
auto postgresql_replica_settings = std::make_unique<MaterializedPostgreSQLSettings>();
if (engine_define->settings)
postgresql_replica_settings->loadFromQuery(*engine_define);
return std::make_shared<DatabaseMaterializedPostgreSQL>(
context, metadata_path, uuid, create.attach,
database_name, postgres_database_name, connection_info,
database_name, configuration.database, connection_info,
std::move(postgresql_replica_settings));
}

View File

@ -39,16 +39,14 @@ DatabasePostgreSQL::DatabasePostgreSQL(
const String & metadata_path_,
const ASTStorage * database_engine_define_,
const String & dbname_,
const String & postgres_dbname_,
const String & postgres_schema_,
const StoragePostgreSQLConfiguration & configuration_,
postgres::PoolWithFailoverPtr pool_,
bool cache_tables_)
: IDatabase(dbname_)
, WithContext(context_->getGlobalContext())
, metadata_path(metadata_path_)
, database_engine_define(database_engine_define_->clone())
, postgres_dbname(postgres_dbname_)
, postgres_schema(postgres_schema_)
, configuration(configuration_)
, pool(std::move(pool_))
, cache_tables(cache_tables_)
{
@ -59,17 +57,17 @@ DatabasePostgreSQL::DatabasePostgreSQL(
String DatabasePostgreSQL::getTableNameForLogs(const String & table_name) const
{
if (postgres_schema.empty())
return fmt::format("{}.{}", postgres_dbname, table_name);
return fmt::format("{}.{}.{}", postgres_dbname, postgres_schema, table_name);
if (configuration.schema.empty())
return fmt::format("{}.{}", configuration.database, table_name);
return fmt::format("{}.{}.{}", configuration.database, configuration.schema, table_name);
}
String DatabasePostgreSQL::formatTableName(const String & table_name) const
{
if (postgres_schema.empty())
if (configuration.schema.empty())
return doubleQuoteString(table_name);
return fmt::format("{}.{}", doubleQuoteString(postgres_schema), doubleQuoteString(table_name));
return fmt::format("{}.{}", doubleQuoteString(configuration.schema), doubleQuoteString(table_name));
}
@ -78,7 +76,7 @@ bool DatabasePostgreSQL::empty() const
std::lock_guard<std::mutex> lock(mutex);
auto connection_holder = pool->get();
auto tables_list = fetchPostgreSQLTablesList(connection_holder->get(), postgres_schema);
auto tables_list = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema);
for (const auto & table_name : tables_list)
if (!detached_or_dropped.count(table_name))
@ -94,7 +92,7 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local
Tables tables;
auto connection_holder = pool->get();
auto table_names = fetchPostgreSQLTablesList(connection_holder->get(), postgres_schema);
auto table_names = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema);
for (const auto & table_name : table_names)
if (!detached_or_dropped.count(table_name))
@ -125,7 +123,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const
"WHERE schemaname != 'pg_catalog' AND {} "
"AND tablename = '{}'",
formatTableName(table_name),
(postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(postgres_schema)),
(configuration.schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(configuration.schema)),
formatTableName(table_name)));
}
catch (pqxx::undefined_table const &)
@ -179,7 +177,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr,
auto storage = StoragePostgreSQL::create(
StorageID(database_name, table_name), pool, table_name,
ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, postgres_schema);
ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, configuration.schema, configuration.on_conflict);
if (cache_tables)
cached_tables[table_name] = storage;
@ -306,7 +304,7 @@ void DatabasePostgreSQL::removeOutdatedTables()
{
std::lock_guard<std::mutex> lock{mutex};
auto connection_holder = pool->get();
auto actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), postgres_schema);
auto actual_tables = fetchPostgreSQLTablesList(connection_holder->get(), configuration.schema);
if (cache_tables)
{

View File

@ -10,7 +10,7 @@
#include <Core/BackgroundSchedulePool.h>
#include <Parsers/ASTCreateQuery.h>
#include <Core/PostgreSQL/PoolWithFailover.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace DB
{
@ -32,8 +32,7 @@ public:
const String & metadata_path_,
const ASTStorage * database_engine_define,
const String & dbname_,
const String & postgres_dbname_,
const String & postgres_schema_,
const StoragePostgreSQLConfiguration & configuration,
postgres::PoolWithFailoverPtr pool_,
bool cache_tables_);
@ -70,8 +69,7 @@ protected:
private:
String metadata_path;
ASTPtr database_engine_define;
String postgres_dbname;
String postgres_schema;
StoragePostgreSQLConfiguration configuration;
postgres::PoolWithFailoverPtr pool;
const bool cache_tables;

View File

@ -2,6 +2,8 @@
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "registerDictionaries.h"
#include <Storages/ExternalDataSourceConfiguration.h>
namespace DB
{
@ -13,19 +15,20 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & root_config_prefix,
Block & sample_block,
ContextPtr,
ContextPtr context,
const std::string & /* default_database */,
bool /* created_from_ddl */)
{
const auto config_prefix = root_config_prefix + ".mongodb";
auto configuration = getExternalDataSourceConfiguration(config, config_prefix, context);
return std::make_unique<MongoDBDictionarySource>(dict_struct,
config.getString(config_prefix + ".uri", ""),
config.getString(config_prefix + ".host", ""),
config.getUInt(config_prefix + ".port", 0),
config.getString(config_prefix + ".user", ""),
config.getString(config_prefix + ".password", ""),
configuration.host,
configuration.port,
configuration.username,
configuration.password,
config.getString(config_prefix + ".method", ""),
config.getString(config_prefix + ".db", ""),
configuration.database,
config.getString(config_prefix + ".collection"),
sample_block);
};

View File

@ -12,6 +12,8 @@
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Processors/Pipe.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace DB
{
@ -32,38 +34,43 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory)
[[maybe_unused]] const std::string & config_prefix,
[[maybe_unused]] Block & sample_block,
[[maybe_unused]] ContextPtr global_context,
const std::string & /* default_database */,
bool /* created_from_ddl */) -> DictionarySourcePtr {
const std::string & /* default_database */,
[[maybe_unused]] bool created_from_ddl) -> DictionarySourcePtr {
#if USE_MYSQL
StreamSettings mysql_input_stream_settings(global_context->getSettingsRef()
, config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false)
, false
, config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss);
StreamSettings mysql_input_stream_settings(
global_context->getSettingsRef(),
config.getBool(config_prefix + ".mysql.close_connection", false) || config.getBool(config_prefix + ".mysql.share_connection", false),
false,
config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss);
auto settings_config_prefix = config_prefix + ".mysql";
auto table = config.getString(settings_config_prefix + ".table", "");
auto where = config.getString(settings_config_prefix + ".where", "");
auto configuration = getExternalDataSourceConfiguration(config, settings_config_prefix, global_context);
auto query = config.getString(settings_config_prefix + ".query", "");
if (query.empty() && table.empty())
if (query.empty() && configuration.table.empty())
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL dictionary source configuration must contain table or query field");
MySQLDictionarySource::Configuration configuration
MySQLDictionarySource::Configuration dictionary_configuration
{
.db = config.getString(settings_config_prefix + ".db", ""),
.table = table,
.db = configuration.database,
.table = configuration.table,
.query = query,
.where = where,
.where = config.getString(settings_config_prefix + ".where", ""),
.invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""),
.update_field = config.getString(settings_config_prefix + ".update_field", ""),
.update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1),
.dont_check_update_time = config.getBool(settings_config_prefix + ".dont_check_update_time", false)
};
auto pool = std::make_shared<mysqlxx::PoolWithFailover>(mysqlxx::PoolFactory::instance().get(config, settings_config_prefix));
std::shared_ptr<mysqlxx::PoolWithFailover> pool;
if (created_from_ddl)
{
std::vector<std::pair<String, UInt16>> addresses{std::make_pair(configuration.host, configuration.port)};
pool = std::make_shared<mysqlxx::PoolWithFailover>(configuration.database, addresses, configuration.username, configuration.password);
}
else
pool = std::make_shared<mysqlxx::PoolWithFailover>(mysqlxx::PoolFactory::instance().get(config, settings_config_prefix));
return std::make_unique<MySQLDictionarySource>(dict_struct, configuration, std::move(pool), sample_block, mysql_input_stream_settings);
return std::make_unique<MySQLDictionarySource>(dict_struct, dictionary_configuration, std::move(pool), sample_block, mysql_input_stream_settings);
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `mysql` is disabled because ClickHouse was built without mysql support.");

View File

@ -11,6 +11,7 @@
#include <DataStreams/PostgreSQLSource.h>
#include "readInvalidateQuery.h"
#include <Interpreters/Context.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#endif
@ -177,22 +178,24 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
ContextPtr global_context,
ContextPtr context,
const std::string & /* default_database */,
bool /* created_from_ddl */) -> DictionarySourcePtr
{
#if USE_LIBPQXX
const auto settings_config_prefix = config_prefix + ".postgresql";
auto pool = std::make_shared<postgres::PoolWithFailover>(
config, settings_config_prefix,
global_context->getSettingsRef().postgresql_connection_pool_size,
global_context->getSettingsRef().postgresql_connection_pool_wait_timeout);
PostgreSQLDictionarySource::Configuration configuration
auto configuration = getExternalDataSourceConfigurationByPriority(config, settings_config_prefix, context);
auto pool = std::make_shared<postgres::PoolWithFailover>(
configuration.replicas_configurations,
context->getSettingsRef().postgresql_connection_pool_size,
context->getSettingsRef().postgresql_connection_pool_wait_timeout);
PostgreSQLDictionarySource::Configuration dictionary_configuration
{
.db = config.getString(fmt::format("{}.db", settings_config_prefix), ""),
.schema = config.getString(fmt::format("{}.schema", settings_config_prefix), ""),
.table = config.getString(fmt::format("{}.table", settings_config_prefix), ""),
.db = configuration.database,
.schema = configuration.schema,
.table = configuration.table,
.query = config.getString(fmt::format("{}.query", settings_config_prefix), ""),
.where = config.getString(fmt::format("{}.where", settings_config_prefix), ""),
.invalidate_query = config.getString(fmt::format("{}.invalidate_query", settings_config_prefix), ""),
@ -200,13 +203,13 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory)
.update_lag = config.getUInt64(fmt::format("{}.update_lag", settings_config_prefix), 1)
};
return std::make_unique<PostgreSQLDictionarySource>(dict_struct, configuration, pool, sample_block);
return std::make_unique<PostgreSQLDictionarySource>(dict_struct, dictionary_configuration, pool, sample_block);
#else
(void)dict_struct;
(void)config;
(void)config_prefix;
(void)sample_block;
(void)global_context;
(void)context;
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
"Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support.");
#endif

View File

@ -3,10 +3,12 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/getLeastSupertype.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnsNumber.h>
@ -373,23 +375,108 @@ public:
size_t getNumberOfArguments() const override { return 2; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
auto first_argument_type = arguments[0].type;
auto second_argument_type = arguments[1].type;
if (!array_type)
throw Exception("First argument for function " + getName() + " must be an array.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(first_argument_type.get());
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(first_argument_type.get());
if (!arguments[1]->onlyNull() && !allowArguments(array_type->getNestedType(), arguments[1]))
DataTypePtr inner_type;
/// If map is first argument only has(map_column, key) function is supported
if constexpr (std::is_same_v<ConcreteAction, HasAction>)
{
if (!array_type && !map_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be an array or map.",
getName());
inner_type = map_type ? map_type->getKeyType() : array_type->getNestedType();
}
else
{
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument for function {} must be an array.",
getName());
inner_type = array_type->getNestedType();
}
if (!second_argument_type->onlyNull() && !allowArguments(inner_type, second_argument_type))
{
const char * first_argument_type_name = map_type ? "map" : "array";
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Types of array and 2nd argument of function `{}` must be identical up to nullability, cardinality, "
"Types of {} and 2nd argument of function `{}` must be identical up to nullability, cardinality, "
"numeric types, or Enum and numeric type. Passed: {} and {}.",
getName(), arguments[0]->getName(), arguments[1]->getName());
first_argument_type_name,
getName(),
first_argument_type->getName(),
second_argument_type->getName());
}
return std::make_shared<DataTypeNumber<ResultType>>();
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
{
if constexpr (std::is_same_v<ConcreteAction, HasAction>)
{
if (isMap(arguments[0].type))
{
auto non_const_map_column = arguments[0].column->convertToFullColumnIfConst();
const auto & map_column = assert_cast<const ColumnMap &>(*non_const_map_column);
const auto & map_array_column = map_column.getNestedColumn();
auto offsets = map_array_column.getOffsetsPtr();
auto keys = map_column.getNestedData().getColumnPtr(0);
auto array_column = ColumnArray::create(std::move(keys), std::move(offsets));
const auto & type_map = assert_cast<const DataTypeMap &>(*arguments[0].type);
auto array_type = std::make_shared<DataTypeArray>(type_map.getKeyType());
auto arguments_copy = arguments;
arguments_copy[0].column = std::move(array_column);
arguments_copy[0].type = std::move(array_type);
arguments_copy[0].name = arguments[0].name;
return executeArrayImpl(arguments_copy, result_type);
}
}
return executeArrayImpl(arguments, result_type);
}
private:
using ResultType = typename ConcreteAction::ResultType;
using ResultColumnType = ColumnVector<ResultType>;
using ResultColumnPtr = decltype(ResultColumnType::create());
using NullMaps = std::pair<const NullMap *, const NullMap *>;
struct ExecutionData
{
const IColumn& left;
const IColumn& right;
const ColumnArray::Offsets& offsets;
ColumnPtr result_column;
NullMaps maps;
ResultColumnPtr result { ResultColumnType::create() };
inline void moveResult() { result_column = std::move(result); }
};
static inline bool allowArguments(const DataTypePtr & inner_type, const DataTypePtr & arg)
{
auto inner_type_decayed = removeNullable(removeLowCardinality(inner_type));
auto arg_decayed = removeNullable(removeLowCardinality(arg));
return ((isNativeNumber(inner_type_decayed) || isEnum(inner_type_decayed)) && isNativeNumber(arg_decayed))
|| getLeastSupertype({inner_type_decayed, arg_decayed});
}
/**
* If one or both arguments passed to this function are nullable,
* we create a new column that contains non-nullable arguments:
@ -404,7 +491,7 @@ public:
* (they are vectors of Fields, which may represent the NULL value),
* they do not require any preprocessing.
*/
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
ColumnPtr executeArrayImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const
{
const ColumnPtr & ptr = arguments[0].column;
@ -419,11 +506,13 @@ public:
if (col_array)
nullable = checkAndGetColumn<ColumnNullable>(col_array->getData());
auto & arg_column = arguments[1].column;
const auto & arg_column = arguments[1].column;
const ColumnNullable * arg_nullable = checkAndGetColumn<ColumnNullable>(*arg_column);
if (!nullable && !arg_nullable)
{
return executeOnNonNullable(arguments, result_type);
}
else
{
/**
@ -483,34 +572,6 @@ public:
}
}
private:
using ResultType = typename ConcreteAction::ResultType;
using ResultColumnType = ColumnVector<ResultType>;
using ResultColumnPtr = decltype(ResultColumnType::create());
using NullMaps = std::pair<const NullMap *, const NullMap *>;
struct ExecutionData
{
const IColumn& left;
const IColumn& right;
const ColumnArray::Offsets& offsets;
ColumnPtr result_column;
NullMaps maps;
ResultColumnPtr result { ResultColumnType::create() };
inline void moveResult() { result_column = std::move(result); }
};
static inline bool allowArguments(const DataTypePtr & array_inner_type, const DataTypePtr & arg)
{
auto inner_type_decayed = removeNullable(removeLowCardinality(array_inner_type));
auto arg_decayed = removeNullable(removeLowCardinality(arg));
return ((isNativeNumber(inner_type_decayed) || isEnum(inner_type_decayed)) && isNativeNumber(arg_decayed))
|| getLeastSupertype({inner_type_decayed, arg_decayed});
}
#define INTEGRAL_TPL_PACK UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64
ColumnPtr executeOnNonNullable(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const

View File

@ -155,61 +155,25 @@ public:
return NameMapContains::name;
}
size_t getNumberOfArguments() const override { return 2; }
size_t getNumberOfArguments() const override { return impl.getNumberOfArguments(); }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override
{
return impl.isSuitableForShortCircuitArgumentsExecution(arguments);
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
if (!map_type)
throw Exception{"First argument for function " + getName() + " must be a map",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto key_type = map_type->getKeyType();
if (!(isNumber(arguments[1].type) && isNumber(key_type))
&& key_type->getName() != arguments[1].type->getName())
throw Exception{"Second argument for function " + getName() + " must be a " + key_type->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
return impl.getReturnTypeImpl(arguments);
}
bool useDefaultImplementationForConstants() const override { return true; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
bool is_const = isColumnConst(*arguments[0].column);
const ColumnMap * col_map = is_const ? checkAndGetColumnConstData<ColumnMap>(arguments[0].column.get()) : checkAndGetColumn<ColumnMap>(arguments[0].column.get());
const DataTypeMap * map_type = checkAndGetDataType<DataTypeMap>(arguments[0].type.get());
if (!col_map || !map_type)
throw Exception{"First argument for function " + getName() + " must be a map", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto key_type = map_type->getKeyType();
const auto & nested_column = col_map->getNestedColumn();
const auto & keys_data = col_map->getNestedData().getColumn(0);
/// Prepare arguments to call arrayIndex for check has the array element.
ColumnPtr column_array = ColumnArray::create(keys_data.getPtr(), nested_column.getOffsetsPtr());
ColumnsWithTypeAndName new_arguments =
{
{
is_const ? ColumnConst::create(std::move(column_array), keys_data.size()) : std::move(column_array),
std::make_shared<DataTypeArray>(key_type),
""
},
arguments[1]
};
return FunctionArrayIndex<HasAction, NameMapContains>().executeImpl(new_arguments, result_type, input_rows_count);
return impl.executeImpl(arguments, result_type, input_rows_count);
}
private:
FunctionArrayIndex<HasAction, NameMapContains> impl;
};

View File

@ -1,11 +1,9 @@
#include <Interpreters/BloomFilter.h>
#include <city.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnLowCardinality.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
@ -126,14 +124,6 @@ DataTypePtr BloomFilter::getPrimitiveType(const DataTypePtr & data_type)
throw Exception("Unexpected type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
}
if (const auto * map_type = typeid_cast<const DataTypeMap *>(data_type.get()))
{
if (!typeid_cast<const DataTypeMap *>(map_type->getKeyType().get()))
return getPrimitiveType(map_type->getKeyType());
else
throw Exception("Unexpected key type " + data_type->getName() + " of bloom filter index for map.", ErrorCodes::BAD_ARGUMENTS);
}
if (const auto * nullable_type = typeid_cast<const DataTypeNullable *>(data_type.get()))
return getPrimitiveType(nullable_type->getNestedType());
@ -148,9 +138,6 @@ ColumnPtr BloomFilter::getPrimitiveColumn(const ColumnPtr & column)
if (const auto * array_col = typeid_cast<const ColumnArray *>(column.get()))
return getPrimitiveColumn(array_col->getDataPtr());
if (const auto * map_col = typeid_cast<const ColumnMap *>(column.get()))
return getPrimitiveColumn(map_col->getNestedData().getColumnPtr(0));
if (const auto * nullable_col = typeid_cast<const ColumnNullable *>(column.get()))
return getPrimitiveColumn(nullable_col->getNestedColumnPtr());

View File

@ -5,7 +5,6 @@
#include <Columns/IColumn.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
@ -124,26 +123,6 @@ struct BloomFilterHash
}
}
if (which.isMap())
{
const auto * map_col = typeid_cast<const ColumnMap *>(column.get());
const auto & keys_data = map_col->getNestedData().getColumn(0);
if (checkAndGetColumn<ColumnNullable>(keys_data))
throw Exception("Unexpected key type " + data_type->getName() + " of bloom filter index.", ErrorCodes::BAD_ARGUMENTS);
const auto & offsets = map_col->getNestedColumn().getOffsets();
limit = offsets[pos + limit - 1] - offsets[pos - 1]; /// PaddedPODArray allows access on index -1.
pos = offsets[pos - 1];
if (limit == 0)
{
auto index_column = ColumnUInt64::create(1);
ColumnUInt64::Container & index_column_vec = index_column->getData();
index_column_vec[0] = 0;
return index_column;
}
}
const ColumnPtr actual_col = BloomFilter::getPrimitiveColumn(column);
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(data_type);

View File

@ -772,7 +772,9 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica(
String shard_path = task.getShardNodePath();
String is_executed_path = fs::path(shard_path) / "executed";
String tries_to_execute_path = fs::path(shard_path) / "tries_to_execute";
zookeeper->createAncestors(fs::path(shard_path) / ""); /* appends "/" at the end of shard_path */
assert(shard_path.starts_with(String(fs::path(task.entry_path) / "shards" / "")));
zookeeper->createIfNotExists(fs::path(task.entry_path) / "shards", "");
zookeeper->createIfNotExists(shard_path, "");
/// Leader replica creates is_executed_path node on successful query execution.
/// We will remove create_shard_flag from zk operations list, if current replica is just waiting for leader to execute the query.

View File

@ -14,18 +14,19 @@ namespace ErrorCodes
/// Mapping from quantile functions for single value to plural
static const std::unordered_map<String, String> quantile_fuse_name_mapping = {
{NameQuantile::name, NameQuantiles::name},
{NameQuantileBFloat16::name, NameQuantilesBFloat16::name},
{NameQuantileBFloat16Weighted::name, NameQuantilesBFloat16Weighted::name},
{NameQuantileDeterministic::name, NameQuantilesDeterministic::name},
{NameQuantileExact::name, NameQuantilesExact::name},
{NameQuantileExactLow::name, NameQuantilesExactLow::name},
{NameQuantileExactHigh::name, NameQuantilesExactHigh::name},
{NameQuantileExactExclusive::name, NameQuantilesExactExclusive::name},
{NameQuantileExactHigh::name, NameQuantilesExactHigh::name},
{NameQuantileExactInclusive::name, NameQuantilesExactInclusive::name},
{NameQuantileExactLow::name, NameQuantilesExactLow::name},
{NameQuantileExactWeighted::name, NameQuantilesExactWeighted::name},
{NameQuantileTiming::name, NameQuantilesTiming::name},
{NameQuantileTimingWeighted::name, NameQuantilesTimingWeighted::name},
{NameQuantileTDigest::name, NameQuantilesTDigest::name},
{NameQuantileTDigestWeighted::name, NameQuantilesTDigestWeighted::name},
{NameQuantileBFloat16::name, NameQuantilesBFloat16::name}
{NameQuantileTiming::name, NameQuantilesTiming::name},
{NameQuantileTimingWeighted::name, NameQuantilesTimingWeighted::name},
};
String GatherFunctionQuantileData::getFusedName(const String & func_name)
@ -54,7 +55,8 @@ void GatherFunctionQuantileData::FuseQuantileAggregatesData::addFuncNode(ASTPtr
bool need_two_args = func->name == NameQuantileDeterministic::name
|| func->name == NameQuantileExactWeighted::name
|| func->name == NameQuantileTimingWeighted::name
|| func->name == NameQuantileTDigestWeighted::name;
|| func->name == NameQuantileTDigestWeighted::name
|| func->name == NameQuantileBFloat16Weighted::name;
if (arguments.size() != (need_two_args ? 2 : 1))
return;

View File

@ -63,7 +63,6 @@ public:
std::shared_ptr<const IExternalLoadable> clone() const override
{
std::cerr << "UserDefinedExecutableFunction::clone " << this << std::endl;
return std::make_shared<UserDefinedExecutableFunction>(configuration, lifetime, process_pool);
}

View File

@ -19,6 +19,41 @@
namespace DB
{
DataTypePtr getCoordinationErrorCodesEnumType()
{
return std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values
{
{"ZOK", static_cast<Int8>(Coordination::Error::ZOK)},
{"ZSYSTEMERROR", static_cast<Int8>(Coordination::Error::ZSYSTEMERROR)},
{"ZRUNTIMEINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZRUNTIMEINCONSISTENCY)},
{"ZDATAINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZDATAINCONSISTENCY)},
{"ZCONNECTIONLOSS", static_cast<Int8>(Coordination::Error::ZCONNECTIONLOSS)},
{"ZMARSHALLINGERROR", static_cast<Int8>(Coordination::Error::ZMARSHALLINGERROR)},
{"ZUNIMPLEMENTED", static_cast<Int8>(Coordination::Error::ZUNIMPLEMENTED)},
{"ZOPERATIONTIMEOUT", static_cast<Int8>(Coordination::Error::ZOPERATIONTIMEOUT)},
{"ZBADARGUMENTS", static_cast<Int8>(Coordination::Error::ZBADARGUMENTS)},
{"ZINVALIDSTATE", static_cast<Int8>(Coordination::Error::ZINVALIDSTATE)},
{"ZAPIERROR", static_cast<Int8>(Coordination::Error::ZAPIERROR)},
{"ZNONODE", static_cast<Int8>(Coordination::Error::ZNONODE)},
{"ZNOAUTH", static_cast<Int8>(Coordination::Error::ZNOAUTH)},
{"ZBADVERSION", static_cast<Int8>(Coordination::Error::ZBADVERSION)},
{"ZNOCHILDRENFOREPHEMERALS", static_cast<Int8>(Coordination::Error::ZNOCHILDRENFOREPHEMERALS)},
{"ZNODEEXISTS", static_cast<Int8>(Coordination::Error::ZNODEEXISTS)},
{"ZNOTEMPTY", static_cast<Int8>(Coordination::Error::ZNOTEMPTY)},
{"ZSESSIONEXPIRED", static_cast<Int8>(Coordination::Error::ZSESSIONEXPIRED)},
{"ZINVALIDCALLBACK", static_cast<Int8>(Coordination::Error::ZINVALIDCALLBACK)},
{"ZINVALIDACL", static_cast<Int8>(Coordination::Error::ZINVALIDACL)},
{"ZAUTHFAILED", static_cast<Int8>(Coordination::Error::ZAUTHFAILED)},
{"ZCLOSING", static_cast<Int8>(Coordination::Error::ZCLOSING)},
{"ZNOTHING", static_cast<Int8>(Coordination::Error::ZNOTHING)},
{"ZSESSIONMOVED", static_cast<Int8>(Coordination::Error::ZSESSIONMOVED)},
});
}
NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes()
{
auto type_enum = std::make_shared<DataTypeEnum8>(
@ -52,36 +87,7 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes()
{"SessionID", static_cast<Int16>(Coordination::OpNum::SessionID)},
});
auto error_enum = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values
{
{"ZOK", static_cast<Int8>(Coordination::Error::ZOK)},
{"ZSYSTEMERROR", static_cast<Int8>(Coordination::Error::ZSYSTEMERROR)},
{"ZRUNTIMEINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZRUNTIMEINCONSISTENCY)},
{"ZDATAINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZDATAINCONSISTENCY)},
{"ZCONNECTIONLOSS", static_cast<Int8>(Coordination::Error::ZCONNECTIONLOSS)},
{"ZMARSHALLINGERROR", static_cast<Int8>(Coordination::Error::ZMARSHALLINGERROR)},
{"ZUNIMPLEMENTED", static_cast<Int8>(Coordination::Error::ZUNIMPLEMENTED)},
{"ZOPERATIONTIMEOUT", static_cast<Int8>(Coordination::Error::ZOPERATIONTIMEOUT)},
{"ZBADARGUMENTS", static_cast<Int8>(Coordination::Error::ZBADARGUMENTS)},
{"ZINVALIDSTATE", static_cast<Int8>(Coordination::Error::ZINVALIDSTATE)},
{"ZAPIERROR", static_cast<Int8>(Coordination::Error::ZAPIERROR)},
{"ZNONODE", static_cast<Int8>(Coordination::Error::ZNONODE)},
{"ZNOAUTH", static_cast<Int8>(Coordination::Error::ZNOAUTH)},
{"ZBADVERSION", static_cast<Int8>(Coordination::Error::ZBADVERSION)},
{"ZNOCHILDRENFOREPHEMERALS", static_cast<Int8>(Coordination::Error::ZNOCHILDRENFOREPHEMERALS)},
{"ZNODEEXISTS", static_cast<Int8>(Coordination::Error::ZNODEEXISTS)},
{"ZNOTEMPTY", static_cast<Int8>(Coordination::Error::ZNOTEMPTY)},
{"ZSESSIONEXPIRED", static_cast<Int8>(Coordination::Error::ZSESSIONEXPIRED)},
{"ZINVALIDCALLBACK", static_cast<Int8>(Coordination::Error::ZINVALIDCALLBACK)},
{"ZINVALIDACL", static_cast<Int8>(Coordination::Error::ZINVALIDACL)},
{"ZAUTHFAILED", static_cast<Int8>(Coordination::Error::ZAUTHFAILED)},
{"ZCLOSING", static_cast<Int8>(Coordination::Error::ZCLOSING)},
{"ZNOTHING", static_cast<Int8>(Coordination::Error::ZNOTHING)},
{"ZSESSIONMOVED", static_cast<Int8>(Coordination::Error::ZSESSIONMOVED)},
});
auto error_enum = getCoordinationErrorCodesEnumType();
auto watch_type_enum = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values

View File

@ -73,4 +73,6 @@ class ZooKeeperLog : public SystemLog<ZooKeeperLogElement>
using SystemLog<ZooKeeperLogElement>::SystemLog;
};
DataTypePtr getCoordinationErrorCodesEnumType();
}

View File

@ -7,15 +7,17 @@
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnLowCardinality.h>
namespace DB
{
@ -50,95 +52,213 @@ void MsgPackVisitor::reset()
info_stack = {};
}
void MsgPackVisitor::insert_integer(UInt64 value) // NOLINT
template <typename InsertFunc>
static bool checkAndInsertNullable(IColumn & column, DataTypePtr type, InsertFunc insert_func)
{
Info & info = info_stack.top();
switch (info.type->getTypeId())
if (type->isNullable())
{
auto & nullable_column = assert_cast<ColumnNullable &>(column);
auto & nested_column = nullable_column.getNestedColumn();
const auto & nested_type = assert_cast<const DataTypeNullable *>(type.get())->getNestedType();
insert_func(nested_column, nested_type);
nullable_column.getNullMapColumn().insertValue(0);
return true;
}
return false;
}
template <typename InsertFunc>
static bool checkAndInsertLowCardinality(IColumn & column, DataTypePtr type, InsertFunc insert_func)
{
if (type->lowCardinality())
{
auto & lc_column = assert_cast<ColumnLowCardinality &>(column);
auto tmp_column = lc_column.getDictionary().getNestedColumn()->cloneEmpty();
auto dict_type = assert_cast<const DataTypeLowCardinality *>(type.get())->getDictionaryType();
insert_func(*tmp_column, dict_type);
lc_column.insertFromFullColumn(*tmp_column, 0);
return true;
}
return false;
}
static void insertInteger(IColumn & column, DataTypePtr type, UInt64 value)
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertInteger(column_, type_, value);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
switch (type->getTypeId())
{
case TypeIndex::UInt8:
{
assert_cast<ColumnUInt8 &>(info.column).insertValue(value);
assert_cast<ColumnUInt8 &>(column).insertValue(value);
break;
}
case TypeIndex::Date: [[fallthrough]];
case TypeIndex::UInt16:
{
assert_cast<ColumnUInt16 &>(info.column).insertValue(value);
assert_cast<ColumnUInt16 &>(column).insertValue(value);
break;
}
case TypeIndex::DateTime: [[fallthrough]];
case TypeIndex::UInt32:
{
assert_cast<ColumnUInt32 &>(info.column).insertValue(value);
assert_cast<ColumnUInt32 &>(column).insertValue(value);
break;
}
case TypeIndex::UInt64:
{
assert_cast<ColumnUInt64 &>(info.column).insertValue(value);
assert_cast<ColumnUInt64 &>(column).insertValue(value);
break;
}
case TypeIndex::Int8:
{
assert_cast<ColumnInt8 &>(info.column).insertValue(value);
assert_cast<ColumnInt8 &>(column).insertValue(value);
break;
}
case TypeIndex::Int16:
{
assert_cast<ColumnInt16 &>(info.column).insertValue(value);
assert_cast<ColumnInt16 &>(column).insertValue(value);
break;
}
case TypeIndex::Int32:
{
assert_cast<ColumnInt32 &>(info.column).insertValue(value);
assert_cast<ColumnInt32 &>(column).insertValue(value);
break;
}
case TypeIndex::Int64:
{
assert_cast<ColumnInt64 &>(info.column).insertValue(value);
assert_cast<ColumnInt64 &>(column).insertValue(value);
break;
}
case TypeIndex::DateTime64:
{
assert_cast<DataTypeDateTime64::ColumnType &>(info.column).insertValue(value);
assert_cast<DataTypeDateTime64::ColumnType &>(column).insertValue(value);
break;
}
default:
throw Exception("Type " + info.type->getName() + " is not supported for MsgPack input format", ErrorCodes::ILLEGAL_COLUMN);
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack integer into column with type {}.", type->getName());
}
}
static void insertString(IColumn & column, DataTypePtr type, const char * value, size_t size)
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertString(column_, type_, value, size);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!isStringOrFixedString(type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack string into column with type {}.", type->getName());
column.insertData(value, size);
}
static void insertFloat32(IColumn & column, DataTypePtr type, Float32 value) // NOLINT
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertFloat32(column_, type_, value);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!WhichDataType(type).isFloat32())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack float32 into column with type {}.", type->getName());
assert_cast<ColumnFloat32 &>(column).insertValue(value);
}
static void insertFloat64(IColumn & column, DataTypePtr type, Float64 value) // NOLINT
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertFloat64(column_, type_, value);
};
if (checkAndInsertNullable(column, type, insert_func) || checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!WhichDataType(type).isFloat64())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack float64 into column with type {}.", type->getName());
assert_cast<ColumnFloat64 &>(column).insertValue(value);
}
static void insertNull(IColumn & column, DataTypePtr type)
{
auto insert_func = [&](IColumn & column_, DataTypePtr type_)
{
insertNull(column_, type_);
};
/// LowCardinality(Nullable(...))
if (checkAndInsertLowCardinality(column, type, insert_func))
return;
if (!type->isNullable())
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack null into non-nullable column with type {}.", type->getName());
assert_cast<ColumnNullable &>(column).insertDefault();
}
bool MsgPackVisitor::visit_positive_integer(UInt64 value) // NOLINT
{
insert_integer(value);
insertInteger(info_stack.top().column, info_stack.top().type, value);
return true;
}
bool MsgPackVisitor::visit_negative_integer(Int64 value) // NOLINT
{
insert_integer(value);
insertInteger(info_stack.top().column, info_stack.top().type, value);
return true;
}
bool MsgPackVisitor::visit_str(const char* value, size_t size) // NOLINT
bool MsgPackVisitor::visit_str(const char * value, size_t size) // NOLINT
{
info_stack.top().column.insertData(value, size);
insertString(info_stack.top().column, info_stack.top().type, value, size);
return true;
}
bool MsgPackVisitor::visit_bin(const char * value, size_t size) // NOLINT
{
insertString(info_stack.top().column, info_stack.top().type, value, size);
return true;
}
bool MsgPackVisitor::visit_float32(Float32 value) // NOLINT
{
assert_cast<ColumnFloat32 &>(info_stack.top().column).insertValue(value);
insertFloat32(info_stack.top().column, info_stack.top().type, value);
return true;
}
bool MsgPackVisitor::visit_float64(Float64 value) // NOLINT
{
assert_cast<ColumnFloat64 &>(info_stack.top().column).insertValue(value);
insertFloat64(info_stack.top().column, info_stack.top().type, value);
return true;
}
bool MsgPackVisitor::visit_boolean(bool value)
{
insertInteger(info_stack.top().column, info_stack.top().type, UInt64(value));
return true;
}
bool MsgPackVisitor::start_array(size_t size) // NOLINT
{
if (!isArray(info_stack.top().type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack array into column with type {}.", info_stack.top().type->getName());
auto nested_type = assert_cast<const DataTypeArray &>(*info_stack.top().type).getNestedType();
ColumnArray & column_array = assert_cast<ColumnArray &>(info_stack.top().column);
ColumnArray::Offsets & offsets = column_array.getOffsets();
@ -154,6 +274,50 @@ bool MsgPackVisitor::end_array() // NOLINT
return true;
}
bool MsgPackVisitor::start_map(uint32_t size) // NOLINT
{
if (!isMap(info_stack.top().type))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert MessagePack map into column with type {}.", info_stack.top().type->getName());
ColumnArray & column_array = assert_cast<ColumnMap &>(info_stack.top().column).getNestedColumn();
ColumnArray::Offsets & offsets = column_array.getOffsets();
offsets.push_back(offsets.back() + size);
return true;
}
bool MsgPackVisitor::start_map_key() // NOLINT
{
auto key_column = assert_cast<ColumnMap &>(info_stack.top().column).getNestedData().getColumns()[0];
auto key_type = assert_cast<const DataTypeMap &>(*info_stack.top().type).getKeyType();
info_stack.push(Info{*key_column, key_type});
return true;
}
bool MsgPackVisitor::end_map_key() // NOLINT
{
info_stack.pop();
return true;
}
bool MsgPackVisitor::start_map_value() // NOLINT
{
auto value_column = assert_cast<ColumnMap &>(info_stack.top().column).getNestedData().getColumns()[1];
auto value_type = assert_cast<const DataTypeMap &>(*info_stack.top().type).getValueType();
info_stack.push(Info{*value_column, value_type});
return true;
}
bool MsgPackVisitor::end_map_value() // NOLINT
{
info_stack.pop();
return true;
}
bool MsgPackVisitor::visit_nil()
{
insertNull(info_stack.top().column, info_stack.top().type);
return true;
}
void MsgPackVisitor::parse_error(size_t, size_t) // NOLINT
{
throw Exception("Error occurred while parsing msgpack data.", ErrorCodes::INCORRECT_DATA);

View File

@ -33,22 +33,27 @@ public:
bool visit_negative_integer(Int64 value);
bool visit_float32(Float32 value);
bool visit_float64(Float64 value);
bool visit_str(const char* value, size_t size);
bool visit_str(const char * value, size_t size);
bool visit_bin(const char * value, size_t size);
bool visit_boolean(bool value);
bool start_array(size_t size);
bool end_array();
bool visit_nil();
bool start_map(uint32_t size);
bool start_map_key();
bool end_map_key();
bool start_map_value();
bool end_map_value();
/// This function will be called if error occurs in parsing
[[noreturn]] void parse_error(size_t parsed_offset, size_t error_offset);
/// Update info_stack
void set_info(IColumn & column, DataTypePtr type);
void insert_integer(UInt64 value);
void reset();
private:
/// Stack is needed to process nested arrays
/// Stack is needed to process arrays and maps
std::stack<Info> info_stack;
};

View File

@ -6,15 +6,18 @@
#include <Common/assert_cast.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnMap.h>
#include <Columns/ColumnLowCardinality.h>
namespace DB
{
@ -91,15 +94,15 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr
case TypeIndex::String:
{
const StringRef & string = assert_cast<const ColumnString &>(column).getDataAt(row_num);
packer.pack_str(string.size);
packer.pack_str_body(string.data, string.size);
packer.pack_bin(string.size);
packer.pack_bin_body(string.data, string.size);
return;
}
case TypeIndex::FixedString:
{
const StringRef & string = assert_cast<const ColumnFixedString &>(column).getDataAt(row_num);
packer.pack_str(string.size);
packer.pack_str_body(string.data, string.size);
packer.pack_bin(string.size);
packer.pack_bin_body(string.data, string.size);
return;
}
case TypeIndex::Array:
@ -132,6 +135,35 @@ void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr
packer.pack_nil();
return;
}
case TypeIndex::Map:
{
const auto & map_column = assert_cast<const ColumnMap &>(column);
const auto & nested_column = map_column.getNestedColumn();
const auto & key_value_columns = map_column.getNestedData().getColumns();
const auto & key_column = key_value_columns[0];
const auto & value_column = key_value_columns[1];
const auto & map_type = assert_cast<const DataTypeMap &>(*data_type);
const auto & offsets = nested_column.getOffsets();
size_t offset = offsets[row_num - 1];
size_t size = offsets[row_num] - offset;
packer.pack_map(size);
for (size_t i = 0; i < size; ++i)
{
serializeField(*key_column, map_type.getKeyType(), offset + i);
serializeField(*value_column, map_type.getValueType(), offset + i);
}
return;
}
case TypeIndex::LowCardinality:
{
const auto & lc_column = assert_cast<const ColumnLowCardinality &>(column);
auto dict_type = assert_cast<const DataTypeLowCardinality *>(data_type.get())->getDictionaryType();
auto dict_column = lc_column.getDictionary().getNestedColumn();
size_t index = lc_column.getIndexAt(row_num);
serializeField(*dict_column, dict_type, index);
return;
}
default:
break;
}

View File

@ -0,0 +1,306 @@
#include "ExternalDataSourceConfiguration.h"
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <IO/WriteBufferFromString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
String ExternalDataSourceConfiguration::toString() const
{
WriteBufferFromOwnString configuration_info;
configuration_info << "username: " << username << "\t";
if (addresses.empty())
{
configuration_info << "host: " << host << "\t";
configuration_info << "port: " << port << "\t";
}
else
{
for (const auto & [replica_host, replica_port] : addresses)
{
configuration_info << "host: " << replica_host << "\t";
configuration_info << "port: " << replica_port << "\t";
}
}
return configuration_info.str();
}
void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration & conf)
{
host = conf.host;
port = conf.port;
username = conf.username;
password = conf.password;
database = conf.database;
table = conf.table;
schema = conf.schema;
}
std::optional<ExternalDataSourceConfig> getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine)
{
if (args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments");
ExternalDataSourceConfiguration configuration;
StorageSpecificArgs non_common_args;
if (const auto * collection = typeid_cast<const ASTIdentifier *>(args[0].get()))
{
const auto & config = context->getConfigRef();
const auto & config_prefix = fmt::format("named_collections.{}", collection->name());
if (!config.has(config_prefix))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection->name());
configuration.host = config.getString(config_prefix + ".host", "");
configuration.port = config.getInt(config_prefix + ".port", 0);
configuration.username = config.getString(config_prefix + ".user", "");
configuration.password = config.getString(config_prefix + ".password", "");
configuration.database = config.getString(config_prefix + ".database", "");
configuration.table = config.getString(config_prefix + ".table", "");
configuration.schema = config.getString(config_prefix + ".schema", "");
if ((args.size() == 1) && (configuration.host.empty() || configuration.port == 0
|| configuration.username.empty() || configuration.password.empty()
|| configuration.database.empty() || (configuration.table.empty() && !is_database_engine)))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Named collection of connection parameters is missing some of the parameters and no key-value arguments are added");
}
for (size_t i = 1; i < args.size(); ++i)
{
if (const auto * ast_function = typeid_cast<const ASTFunction *>(args[i].get()))
{
const auto * args_expr = assert_cast<const ASTExpressionList *>(ast_function->arguments.get());
auto function_args = args_expr->children;
if (function_args.size() != 2)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument");
auto arg_name = function_args[0]->as<ASTIdentifier>()->name();
auto arg_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as<ASTLiteral>()->value;
if (arg_name == "host")
configuration.host = arg_value.safeGet<String>();
else if (arg_name == "port")
configuration.port = arg_value.safeGet<UInt64>();
else if (arg_name == "user")
configuration.username = arg_value.safeGet<String>();
else if (arg_name == "password")
configuration.password = arg_value.safeGet<String>();
else if (arg_name == "database")
configuration.database = arg_value.safeGet<String>();
else if (arg_name == "table")
configuration.table = arg_value.safeGet<String>();
else if (arg_name == "schema")
configuration.schema = arg_value.safeGet<String>();
else
non_common_args.emplace_back(std::make_pair(arg_name, arg_value));
}
else
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument");
}
}
ExternalDataSourceConfig source_config{ .configuration = configuration, .specific_args = non_common_args };
return source_config;
}
return std::nullopt;
}
ExternalDataSourceConfiguration getExternalDataSourceConfiguration(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context)
{
ExternalDataSourceConfiguration configuration;
auto collection_name = dict_config.getString(dict_config_prefix + ".name", "");
if (!collection_name.empty())
{
const auto & config = context->getConfigRef();
const auto & config_prefix = fmt::format("named_collections.{}", collection_name);
if (!config.has(config_prefix))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name);
configuration.host = dict_config.getString(dict_config_prefix + ".host", config.getString(config_prefix + ".host", ""));
configuration.port = dict_config.getInt(dict_config_prefix + ".port", config.getUInt(config_prefix + ".port", 0));
configuration.username = dict_config.getString(dict_config_prefix + ".user", config.getString(config_prefix + ".user", ""));
configuration.password = dict_config.getString(dict_config_prefix + ".password", config.getString(config_prefix + ".password", ""));
configuration.database = dict_config.getString(dict_config_prefix + ".db", config.getString(config_prefix + ".database", ""));
configuration.table = dict_config.getString(dict_config_prefix + ".table", config.getString(config_prefix + ".table", ""));
configuration.schema = dict_config.getString(dict_config_prefix + ".schema", config.getString(config_prefix + ".schema", ""));
if (configuration.host.empty() || configuration.port == 0 || configuration.username.empty() || configuration.password.empty()
|| configuration.database.empty() || configuration.table.empty())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Named collection of connection parameters is missing some of the parameters and dictionary parameters are added");
}
}
else
{
configuration.host = dict_config.getString(dict_config_prefix + ".host", "");
configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0);
configuration.username = dict_config.getString(dict_config_prefix + ".user", "");
configuration.password = dict_config.getString(dict_config_prefix + ".password", "");
configuration.database = dict_config.getString(dict_config_prefix + ".db", "");
configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), "");
configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), "");
}
return configuration;
}
ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context)
{
auto common_configuration = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context);
ExternalDataSourcesByPriority configuration
{
.database = common_configuration.database,
.table = common_configuration.table,
.schema = common_configuration.schema,
.replicas_configurations = {}
};
if (dict_config.has(dict_config_prefix + ".replica"))
{
Poco::Util::AbstractConfiguration::Keys config_keys;
dict_config.keys(dict_config_prefix, config_keys);
for (const auto & config_key : config_keys)
{
if (config_key.starts_with("replica"))
{
ExternalDataSourceConfiguration replica_configuration(common_configuration);
String replica_name = dict_config_prefix + "." + config_key;
size_t priority = dict_config.getInt(replica_name + ".priority", 0);
replica_configuration.host = dict_config.getString(replica_name + ".host", common_configuration.host);
replica_configuration.port = dict_config.getUInt(replica_name + ".port", common_configuration.port);
replica_configuration.username = dict_config.getString(replica_name + ".user", common_configuration.username);
replica_configuration.password = dict_config.getString(replica_name + ".password", common_configuration.password);
if (replica_configuration.host.empty() || replica_configuration.port == 0
|| replica_configuration.username.empty() || replica_configuration.password.empty())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Named collection of connection parameters is missing some of the parameters and no other dictionary parameters are added");
}
configuration.replicas_configurations[priority].emplace_back(replica_configuration);
}
}
}
else
{
configuration.replicas_configurations[0].emplace_back(common_configuration);
}
return configuration;
}
void URLBasedDataSourceConfiguration::set(const URLBasedDataSourceConfiguration & conf)
{
url = conf.url;
format = conf.format;
compression_method = conf.compression_method;
structure = conf.structure;
}
std::optional<URLBasedDataSourceConfig> getURLBasedDataSourceConfiguration(const ASTs & args, ContextPtr context)
{
if (args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments");
URLBasedDataSourceConfiguration configuration;
StorageSpecificArgs non_common_args;
if (const auto * collection = typeid_cast<const ASTIdentifier *>(args[0].get()))
{
const auto & config = context->getConfigRef();
auto config_prefix = fmt::format("named_collections.{}", collection->name());
if (!config.has(config_prefix))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection->name());
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
for (const auto & key : keys)
{
if (key == "url")
{
configuration.url = config.getString(config_prefix + ".url", "");
}
else if (key == "headers")
{
Poco::Util::AbstractConfiguration::Keys header_keys;
config.keys(config_prefix + ".headers", header_keys);
for (const auto & header : header_keys)
{
const auto header_prefix = config_prefix + ".headers." + header;
configuration.headers.emplace_back(std::make_pair(config.getString(header_prefix + ".name"), config.getString(header_prefix + ".value")));
}
}
else
non_common_args.emplace_back(std::make_pair(key, config.getString(config_prefix + '.' + key)));
}
for (size_t i = 1; i < args.size(); ++i)
{
if (const auto * ast_function = typeid_cast<const ASTFunction *>(args[i].get()))
{
const auto * args_expr = assert_cast<const ASTExpressionList *>(ast_function->arguments.get());
auto function_args = args_expr->children;
if (function_args.size() != 2)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument");
auto arg_name = function_args[0]->as<ASTIdentifier>()->name();
auto arg_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as<ASTLiteral>()->value;
if (arg_name == "url")
configuration.url = arg_value.safeGet<String>();
else if (arg_name == "format")
configuration.format = arg_value.safeGet<String>();
else if (arg_name == "compression_method")
configuration.compression_method = arg_value.safeGet<String>();
else if (arg_name == "structure")
configuration.structure = arg_value.safeGet<String>();
else
non_common_args.emplace_back(std::make_pair(arg_name, arg_value));
}
else
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument");
}
}
if (configuration.url.empty() || configuration.format.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Storage requires {}", configuration.url.empty() ? "url" : "format");
URLBasedDataSourceConfig source_config{ .configuration = configuration, .specific_args = non_common_args };
return source_config;
}
return std::nullopt;
}
}

View File

@ -0,0 +1,115 @@
#pragma once
#include <Parsers/ASTIdentifier.h>
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
struct ExternalDataSourceConfiguration
{
String host;
UInt16 port = 0;
String username;
String password;
String database;
String table;
String schema;
std::vector<std::pair<String, UInt16>> addresses; /// Failover replicas.
String toString() const;
void set(const ExternalDataSourceConfiguration & conf);
};
using ExternalDataSourceConfigurationPtr = std::shared_ptr<ExternalDataSourceConfiguration>;
struct StoragePostgreSQLConfiguration : ExternalDataSourceConfiguration
{
String on_conflict;
};
struct StorageMySQLConfiguration : ExternalDataSourceConfiguration
{
bool replace_query = false;
String on_duplicate_clause;
};
struct StorageMongoDBConfiguration : ExternalDataSourceConfiguration
{
String collection;
String options;
};
using StorageSpecificArgs = std::vector<std::pair<String, DB::Field>>;
struct ExternalDataSourceConfig
{
ExternalDataSourceConfiguration configuration;
StorageSpecificArgs specific_args;
};
/* If there is a storage engine's configuration specified in the named_collections,
* this function returns valid for usage ExternalDataSourceConfiguration struct
* otherwise std::nullopt is returned.
*
* If any configuration options are provided as key-value engine arguments, they will override
* configuration values, i.e. ENGINE = PostgreSQL(postgresql_configuration, database = 'postgres_database');
*
* Any key-value engine argument except common (`host`, `port`, `username`, `password`, `database`)
* is returned in EngineArgs struct.
*/
std::optional<ExternalDataSourceConfig> getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine = false);
ExternalDataSourceConfiguration getExternalDataSourceConfiguration(
const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context);
/// Highest priority is 0, the bigger the number in map, the less the priority.
using ExternalDataSourcesConfigurationByPriority = std::map<size_t, std::vector<ExternalDataSourceConfiguration>>;
struct ExternalDataSourcesByPriority
{
String database;
String table;
String schema;
ExternalDataSourcesConfigurationByPriority replicas_configurations;
};
ExternalDataSourcesByPriority
getExternalDataSourceConfigurationByPriority(const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context);
struct URLBasedDataSourceConfiguration
{
String url;
String format;
String compression_method = "auto";
String structure;
std::vector<std::pair<String, Field>> headers;
void set(const URLBasedDataSourceConfiguration & conf);
};
struct StorageS3Configuration : URLBasedDataSourceConfiguration
{
String access_key_id;
String secret_access_key;
};
struct URLBasedDataSourceConfig
{
URLBasedDataSourceConfiguration configuration;
StorageSpecificArgs specific_args;
};
std::optional<URLBasedDataSourceConfig> getURLBasedDataSourceConfiguration(const ASTs & args, ContextPtr context);
}

View File

@ -71,7 +71,7 @@ public:
void scheduleMoveTask(ExecutableTaskPtr move_task);
/// Just call finish
virtual ~BackgroundJobsAssignee();
~BackgroundJobsAssignee();
BackgroundJobsAssignee(
MergeTreeData & data_,

View File

@ -114,6 +114,7 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item)
/// This is significant to order the destructors.
item->task.reset();
item->is_done.set();
return;
}
@ -149,6 +150,7 @@ void MergeTreeBackgroundExecutor::routine(TaskRuntimeDataPtr item)
/// The thread that shutdowns storage will scan queues in order to find some tasks to wait for, but will find nothing.
/// So, the destructor of a task and the destructor of a storage will be executed concurrently.
item->task.reset();
item->is_done.set();
}
}
@ -176,13 +178,7 @@ void MergeTreeBackgroundExecutor::threadFunction()
active.push_back(item);
}
routine(item);
/// When storage shutdowns it will wait until all related background tasks
/// are finished, because they may want to interact with its fields
/// and this will cause segfault.
if (item->is_currently_deleting)
item->is_done.set();
routine(std::move(item));
}
catch (...)
{

View File

@ -128,7 +128,7 @@ private:
ExecutableTaskPtr task;
CurrentMetrics::Increment increment;
std::atomic_bool is_currently_deleting{false};
bool is_currently_deleting{false};
/// Actually autoreset=false is needed only for unit test
/// where multiple threads could remove tasks corresponding to the same storage
/// This scenario in not possible in reality.

View File

@ -46,20 +46,29 @@ MergeTreeIndexGranulePtr MergeTreeIndexBloomFilter::createIndexGranule() const
bool MergeTreeIndexBloomFilter::mayBenefitFromIndexForIn(const ASTPtr & node) const
{
const String & column_name = node->getColumnName();
Names required_columns = index.expression->getRequiredColumns();
NameSet required_columns_set(required_columns.begin(), required_columns.end());
for (const auto & cname : index.column_names)
if (column_name == cname)
std::vector<ASTPtr> nodes_to_check;
nodes_to_check.emplace_back(node);
while (!nodes_to_check.empty())
{
auto node_to_check = nodes_to_check.back();
nodes_to_check.pop_back();
const auto & column_name = node_to_check->getColumnName();
if (required_columns_set.find(column_name) != required_columns_set.end())
return true;
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
{
for (const auto & children : func->arguments->children)
if (mayBenefitFromIndexForIn(children))
return true;
if (const auto * function = typeid_cast<const ASTFunction *>(node_to_check.get()))
{
auto & function_arguments_children = function->arguments->children;
nodes_to_check.insert(nodes_to_check.end(), function_arguments_children.begin(), function_arguments_children.end());
}
}
return false;
return true;
}
MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator() const

View File

@ -276,7 +276,9 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B
if (functionIsInOrGlobalInOperator(function->name))
{
if (const auto & prepared_set = getPreparedSet(arguments[1]))
auto prepared_set = getPreparedSet(arguments[1]);
if (prepared_set)
{
if (traverseASTIn(function->name, arguments[0], prepared_set, out))
maybe_useful = true;
@ -285,6 +287,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B
else if (function->name == "equals" ||
function->name == "notEquals" ||
function->name == "has" ||
function->name == "mapContains" ||
function->name == "indexOf" ||
function->name == "hasAny" ||
function->name == "hasAll")
@ -308,14 +311,22 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B
}
bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
const String & function_name, const ASTPtr & key_ast, const SetPtr & prepared_set, RPNElement & out)
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
RPNElement & out)
{
const auto prepared_info = getPreparedSetInfo(prepared_set);
return traverseASTIn(function_name, key_ast, prepared_info.type, prepared_info.column, out);
return traverseASTIn(function_name, key_ast, prepared_set, prepared_info.type, prepared_info.column, out);
}
bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
const String & function_name, const ASTPtr & key_ast, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out)
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
const DataTypePtr & type,
const ColumnPtr & column,
RPNElement & out)
{
if (header.has(key_ast->getColumnName()))
{
@ -352,10 +363,83 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
const auto & sub_data_types = tuple_data_type->getElements();
for (size_t index = 0; index < arguments.size(); ++index)
match_with_subtype |= traverseASTIn(function_name, arguments[index], sub_data_types[index], sub_columns[index], out);
match_with_subtype |= traverseASTIn(function_name, arguments[index], nullptr, sub_data_types[index], sub_columns[index], out);
return match_with_subtype;
}
if (function->name == "arrayElement")
{
/** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map
* we return default value for arrayElement.
*
* We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists.
*/
if (!prepared_set)
return false;
auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst();
ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" };
ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check};
auto set_contains_default_value_predicate_column = prepared_set->execute(default_columns_with_type_to_check, false /*negative*/);
const auto & set_contains_default_value_predicate_column_typed = assert_cast<const ColumnUInt8 &>(*set_contains_default_value_predicate_column);
bool set_contain_default_value = set_contains_default_value_predicate_column_typed.getData()[0];
if (set_contain_default_value)
return false;
const auto & col_name = assert_cast<ASTIdentifier *>(function->arguments.get()->children[0].get())->name();
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
auto map_values_index_column_name = fmt::format("mapValues({})", col_name);
if (header.has(map_keys_index_column_name))
{
/// For mapKeys we serialize key argument with bloom filter
auto & argument = function->arguments.get()->children[1];
if (const auto * literal = argument->as<ASTLiteral>())
{
size_t position = header.getPositionByName(map_keys_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key)));
return true;
}
else
{
return false;
}
}
else if (header.has(map_values_index_column_name))
{
/// For mapValues we serialize set with bloom filter
size_t row_size = column->size();
size_t position = header.getPositionByName(map_values_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
const auto & array_type = assert_cast<const DataTypeArray &>(*index_type);
const auto & array_nested_type = array_type.getNestedType();
const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, array_nested_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(array_nested_type, converted_column, 0, row_size)));
}
else
{
return false;
}
if (function_name == "in" || function_name == "globalIn")
out.function = RPNElement::FUNCTION_IN;
if (function_name == "notIn" || function_name == "globalNotIn")
out.function = RPNElement::FUNCTION_NOT_IN;
return true;
}
}
return false;
@ -420,7 +504,12 @@ static bool indexOfCanUseBloomFilter(const ASTPtr & parent)
bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out, const ASTPtr & parent)
const String & function_name,
const ASTPtr & key_ast,
const DataTypePtr & value_type,
const Field & value_field,
RPNElement & out,
const ASTPtr & parent)
{
if (header.has(key_ast->getColumnName()))
{
@ -488,6 +577,29 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
return true;
}
if (function_name == "mapContains" || function_name == "has")
{
const auto & col_name = assert_cast<ASTIdentifier *>(key_ast.get())->name();
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
if (!header.has(map_keys_index_column_name))
return false;
size_t position = header.getPositionByName(map_keys_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
const auto * array_type = typeid_cast<const DataTypeArray *>(index_type.get());
if (!array_type)
return false;
out.function = RPNElement::FUNCTION_HAS;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(array_type->getNestedType());
Field converted_field = convertFieldToType(value_field, *actual_type, value_type.get());
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), converted_field)));
return true;
}
if (const auto * function = key_ast->as<ASTFunction>())
{
WhichDataType which(value_type);
@ -512,33 +624,38 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals(
if (function->name == "arrayElement")
{
/** Try to parse arrayElement for mapKeys index.
* It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map
* we return default value for arrayElement.
*
* We cannot skip keys that does not exist in map if comparison is with default type value because
* that way we skip necessary granules where map key does not exists.
*/
if (value_field == value_type->getDefault())
return false;
const auto & col_name = assert_cast<ASTIdentifier *>(function->arguments.get()->children[0].get())->name();
if (header.has(col_name))
auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name);
if (!header.has(map_keys_index_column_name))
return false;
size_t position = header.getPositionByName(map_keys_index_column_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS;
auto & argument = function->arguments.get()->children[1];
if (const auto * literal = argument->as<ASTLiteral>())
{
size_t position = header.getPositionByName(col_name);
const DataTypePtr & index_type = header.getByPosition(position).type;
const auto * map_type = typeid_cast<const DataTypeMap *>(index_type.get());
if (map_type)
{
out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS;
auto & argument = function->arguments.get()->children[1];
if (const auto * literal = argument->as<ASTLiteral>())
{
auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key)));
}
else
{
return false;
}
return true;
}
auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key)));
return true;
}
return false;
}
}

View File

@ -70,13 +70,27 @@ private:
bool traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent);
bool traverseASTIn(const String & function_name, const ASTPtr & key_ast, const SetPtr & prepared_set, RPNElement & out);
bool traverseASTIn(
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
RPNElement & out);
bool traverseASTIn(
const String & function_name, const ASTPtr & key_ast, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out);
const String & function_name,
const ASTPtr & key_ast,
const SetPtr & prepared_set,
const DataTypePtr & type,
const ColumnPtr & column,
RPNElement & out);
bool traverseASTEquals(
const String & function_name, const ASTPtr & key_ast, const DataTypePtr & value_type, const Field & value_field, RPNElement & out, const ASTPtr & parent);
const String & function_name,
const ASTPtr & key_ast,
const DataTypePtr & value_type,
const Field & value_field,
RPNElement & out,
const ASTPtr & parent);
};
}

View File

@ -3,6 +3,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/UTF8Helpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/ExpressionActions.h>
@ -155,13 +156,40 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos,
for (size_t col = 0; col < index_columns.size(); ++col)
{
const auto & column = block.getByName(index_columns[col]).column;
for (size_t i = 0; i < rows_read; ++i)
const auto & column_with_type = block.getByName(index_columns[col]);
const auto & column = column_with_type.column;
size_t current_position = *pos;
if (isArray(column_with_type.type))
{
auto ref = column->getDataAt(*pos + i);
columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]);
const auto & column_array = assert_cast<const ColumnArray &>(*column);
const auto & column_offsets = column_array.getOffsets();
const auto & column_key = column_array.getData();
for (size_t i = 0; i < rows_read; ++i)
{
size_t element_start_row = column_offsets[current_position - 1];
size_t elements_size = column_offsets[current_position] - element_start_row;
for (size_t row_num = 0; row_num < elements_size; row_num++)
{
auto ref = column_key.getDataAt(element_start_row + row_num);
columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]);
}
current_position += 1;
}
}
else
{
for (size_t i = 0; i < rows_read; ++i)
{
auto ref = column->getDataAt(current_position + i);
columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]);
}
}
}
granule->has_elems = true;
*pos += rows_read;
}
@ -202,6 +230,7 @@ bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const
}
else if (element.function == RPNElement::FUNCTION_EQUALS
|| element.function == RPNElement::FUNCTION_NOT_EQUALS
|| element.function == RPNElement::FUNCTION_HAS
|| element.function == RPNElement::FUNCTION_IN
|| element.function == RPNElement::FUNCTION_NOT_IN
|| element.function == RPNElement::FUNCTION_MULTI_SEARCH
@ -251,7 +280,8 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx
rpn_stack.emplace_back(true, true);
}
else if (element.function == RPNElement::FUNCTION_EQUALS
|| element.function == RPNElement::FUNCTION_NOT_EQUALS)
|| element.function == RPNElement::FUNCTION_NOT_EQUALS
|| element.function == RPNElement::FUNCTION_HAS)
{
rpn_stack.emplace_back(granule->bloom_filters[element.key_column].contains(*element.bloom_filter), true);
@ -378,6 +408,15 @@ bool MergeTreeConditionFullText::atomFromAST(
else if (!token_extractor->supportLike() && (func_name == "like" || func_name == "notLike"))
return false;
if (func_name == "has")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_HAS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
stringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
if (func_name == "notEquals")
{
out.key_column = key_column_num;
@ -837,10 +876,18 @@ MergeTreeIndexPtr bloomFilterIndexCreator(
void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/)
{
for (const auto & data_type : index.data_types)
for (const auto & index_data_type : index.data_types)
{
if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString)
throw Exception("Bloom filter index can be used only with `String` or `FixedString` column.", ErrorCodes::INCORRECT_QUERY);
WhichDataType data_type(index_data_type);
if (data_type.isArray())
{
const auto & array_type = assert_cast<const DataTypeArray &>(*index_data_type);
data_type = WhichDataType(array_type.getNestedType());
}
if (!data_type.isString() && !data_type.isFixedString())
throw Exception("Bloom filter index can be used only with `String`, `FixedString` column or Array with `String` or `FixedString` values column.", ErrorCodes::INCORRECT_QUERY);
}
if (index.type == NgramTokenExtractor::getName())

View File

@ -112,6 +112,7 @@ private:
/// Atoms of a Boolean expression.
FUNCTION_EQUALS,
FUNCTION_NOT_EQUALS,
FUNCTION_HAS,
FUNCTION_IN,
FUNCTION_NOT_IN,
FUNCTION_MULTI_SEARCH,

View File

@ -1,10 +1,11 @@
#include "StorageMaterializedPostgreSQL.h"
#if USE_LIBPQXX
#include <common/logger_useful.h>
#include <Common/Macros.h>
#include <Core/Settings.h>
#include <Common/parseAddress.h>
#include <Common/assert_cast.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeArray.h>
@ -20,8 +21,8 @@
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Storages/StorageFactory.h>
#include <common/logger_useful.h>
#include <Storages/ReadFinalForExternalReplicaStorage.h>
#include <Storages/StoragePostgreSQL.h>
#include <Core/PostgreSQL/Connection.h>
@ -30,7 +31,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
@ -471,22 +471,6 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory)
{
auto creator_fn = [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
bool has_settings = args.storage_def->settings;
auto postgresql_replication_settings = std::make_unique<MaterializedPostgreSQLSettings>();
if (has_settings)
postgresql_replication_settings->loadFromQuery(*args.storage_def);
if (engine_args.size() != 5)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage MaterializedPostgreSQL requires 5 parameters: "
"PostgreSQL('host:port', 'database', 'table', 'username', 'password'. Got {}",
engine_args.size());
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getContext());
StorageInMemoryMetadata metadata;
metadata.setColumns(args.columns);
metadata.setConstraints(args.constraints);
@ -502,20 +486,19 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory)
else
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.getContext());
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 5432);
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
/// No connection is made here, see Storages/PostgreSQL/PostgreSQLConnection.cpp
auto configuration = StoragePostgreSQL::getConfiguration(args.engine_args, args.getContext());
auto connection_info = postgres::formatConnectionString(
remote_database,
parsed_host_port.first,
parsed_host_port.second,
engine_args[3]->as<ASTLiteral &>().value.safeGet<String>(),
engine_args[4]->as<ASTLiteral &>().value.safeGet<String>());
configuration.database, configuration.host, configuration.port,
configuration.username, configuration.password);
bool has_settings = args.storage_def->settings;
auto postgresql_replication_settings = std::make_unique<MaterializedPostgreSQLSettings>();
if (has_settings)
postgresql_replication_settings->loadFromQuery(*args.storage_def);
return StorageMaterializedPostgreSQL::create(
args.table_id, args.attach, remote_database, remote_table, connection_info,
args.table_id, args.attach, configuration.database, configuration.table, connection_info,
metadata, args.getContext(),
std::move(postgresql_replication_settings));
};

View File

@ -15,6 +15,7 @@
#include <Storages/MySQL/MySQLSettings.h>
#include <Storages/StoragePostgreSQL.h>
#include <Storages/StorageURL.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <common/logger_useful.h>
@ -31,10 +32,7 @@ StorageExternalDistributed::StorageExternalDistributed(
const StorageID & table_id_,
ExternalStorageEngine table_engine,
const String & cluster_description,
const String & remote_database,
const String & remote_table,
const String & username,
const String & password,
const ExternalDataSourceConfiguration & configuration,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
@ -66,15 +64,16 @@ StorageExternalDistributed::StorageExternalDistributed(
addresses = parseRemoteDescriptionForExternalDatabase(shard_description, max_addresses, 3306);
mysqlxx::PoolWithFailover pool(
remote_database,
configuration.database,
addresses,
username, password);
configuration.username,
configuration.password);
shard = StorageMySQL::create(
table_id_,
std::move(pool),
remote_database,
remote_table,
configuration.database,
configuration.table,
/* replace_query = */ false,
/* on_duplicate_clause = */ "",
columns_,
@ -90,15 +89,16 @@ StorageExternalDistributed::StorageExternalDistributed(
case ExternalStorageEngine::PostgreSQL:
{
addresses = parseRemoteDescriptionForExternalDatabase(shard_description, max_addresses, 5432);
StoragePostgreSQLConfiguration postgres_conf;
postgres_conf.set(configuration);
postgres_conf.addresses = addresses;
auto pool = std::make_shared<postgres::PoolWithFailover>(
remote_database,
addresses,
username, password,
postgres_conf,
context->getSettingsRef().postgresql_connection_pool_size,
context->getSettingsRef().postgresql_connection_pool_wait_timeout);
shard = StoragePostgreSQL::create(table_id_, std::move(pool), remote_table, columns_, constraints_, String{});
shard = StoragePostgreSQL::create(table_id_, std::move(pool), configuration.table, columns_, constraints_, String{});
break;
}
#endif
@ -113,13 +113,10 @@ StorageExternalDistributed::StorageExternalDistributed(
}
#else
(void)table_engine;
(void)remote_database;
(void)remote_table;
(void)username;
(void)password;
(void)shards_descriptions;
(void)configuration;
(void)cluster_description;
(void)addresses;
(void)table_engine;
#endif
}
@ -207,64 +204,113 @@ void registerStorageExternalDistributed(StorageFactory & factory)
factory.registerStorage("ExternalDistributed", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
if (engine_args.size() < 2)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine ExternalDistributed must have at least 2 arguments: engine_name, named_collection and/or description");
if (engine_args.size() != 6)
throw Exception(
"Storage MySQLiDistributed requires 5 parameters: ExternalDistributed('engine_name', 'cluster_description', database, table, 'user', 'password').",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
const String & engine_name = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
const String & addresses_description = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
auto engine_name = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
StorageExternalDistributed::ExternalStorageEngine table_engine;
if (engine_name == "URL")
{
table_engine = StorageExternalDistributed::ExternalStorageEngine::URL;
else if (engine_name == "MySQL")
table_engine = StorageExternalDistributed::ExternalStorageEngine::MySQL;
else if (engine_name == "PostgreSQL")
table_engine = StorageExternalDistributed::ExternalStorageEngine::PostgreSQL;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"External storage engine {} is not supported for StorageExternalDistributed. Supported engines are: MySQL, PostgreSQL, URL",
engine_name);
ASTs inner_engine_args(engine_args.begin() + 1, engine_args.end());
String cluster_description;
if (engine_name == "URL")
{
URLBasedDataSourceConfiguration configuration;
if (auto named_collection = getURLBasedDataSourceConfiguration(inner_engine_args, args.getLocalContext()))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
for (const auto & [name, value] : storage_specific_args)
{
if (name == "description")
cluster_description = value.safeGet<String>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unknown key-value argument {} for table engine URL", name);
}
if (cluster_description.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Engine ExternalDistribued must have `description` key-value argument or named collection parameter");
}
else
{
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
cluster_description = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.format = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
configuration.compression_method = "auto";
if (engine_args.size() == 4)
configuration.compression_method = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
}
const String & format_name = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
String compression_method = "auto";
if (engine_args.size() == 4)
compression_method = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
auto format_settings = StorageURL::getFormatSettingsFromArgs(args);
return StorageExternalDistributed::create(
addresses_description,
cluster_description,
args.table_id,
format_name,
configuration.format,
format_settings,
compression_method,
configuration.compression_method,
args.columns,
args.constraints,
args.getContext());
}
else
{
if (engine_name == "MySQL")
table_engine = StorageExternalDistributed::ExternalStorageEngine::MySQL;
else if (engine_name == "PostgreSQL")
table_engine = StorageExternalDistributed::ExternalStorageEngine::PostgreSQL;
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"External storage engine {} is not supported for StorageExternalDistributed. Supported engines are: MySQL, PostgreSQL, URL",
engine_name);
ExternalDataSourceConfiguration configuration;
if (auto named_collection = getExternalDataSourceConfiguration(inner_engine_args, args.getLocalContext()))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
for (const auto & [name, value] : storage_specific_args)
{
if (name == "description")
cluster_description = value.safeGet<String>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unknown key-value argument {} for table function URL", name);
}
if (cluster_description.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Engine ExternalDistribued must have `description` key-value argument or named collection parameter");
}
else
{
if (engine_args.size() != 6)
throw Exception(
"Storage ExternalDistributed requires 5 parameters: "
"ExternalDistributed('engine_name', 'cluster_description', 'database', 'table', 'user', 'password').",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
cluster_description = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.database = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
configuration.table = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
configuration.username = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
configuration.password = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
}
const String & remote_database = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
const String & remote_table = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
const String & username = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
const String & password = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
return StorageExternalDistributed::create(
args.table_id,
table_engine,
addresses_description,
remote_database,
remote_table,
username,
password,
cluster_description,
configuration,
args.columns,
args.constraints,
args.comment,

View File

@ -11,11 +11,13 @@
namespace DB
{
struct ExternalDataSourceConfiguration;
/// Storages MySQL and PostgreSQL use ConnectionPoolWithFailover and support multiple replicas.
/// This class unites multiple storages with replicas into multiple shards with replicas.
/// A query to external database is passed to one replica on each shard, the result is united.
/// Replicas on each shard have the same priority, traversed replicas are moved to the end of the queue.
/// TODO: try `load_balancing` setting for replicas priorities same way as for table function `remote`
/// Similar approach is used for URL storage.
class StorageExternalDistributed final : public shared_ptr_helper<StorageExternalDistributed>, public DB::IStorage
{
friend struct shared_ptr_helper<StorageExternalDistributed>;
@ -44,10 +46,7 @@ protected:
const StorageID & table_id_,
ExternalStorageEngine table_engine,
const String & cluster_description,
const String & remote_database_,
const String & remote_table_,
const String & username,
const String & password,
const ExternalDataSourceConfiguration & configuration,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,

View File

@ -24,6 +24,7 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int MONGODB_CANNOT_AUTHENTICATE;
extern const int BAD_ARGUMENTS;
}
StorageMongoDB::StorageMongoDB(
@ -102,42 +103,72 @@ Pipe StorageMongoDB::read(
return Pipe(std::make_shared<MongoDBSource>(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true));
}
void registerStorageMongoDB(StorageFactory & factory)
{
factory.registerStorage("MongoDB", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, ContextPtr context)
{
StorageMongoDBConfiguration configuration;
if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
if (arg_name == "collection")
configuration.collection = arg_value.safeGet<String>();
else if (arg_name == "options")
configuration.options = arg_value.safeGet<String>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unexpected key-value argument."
"Got: {}, but expected one of:"
"host, port, username, password, database, table, options.", arg_name);
}
}
else
{
if (engine_args.size() < 5 || engine_args.size() > 6)
throw Exception(
"Storage MongoDB requires from 5 to 6 parameters: MongoDB('host:port', database, collection, 'user', 'password' [, 'options']).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
/// 27017 is the default MongoDB port.
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 27017);
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
const String & collection = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
const String & username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
String options;
configuration.host = parsed_host_port.first;
configuration.port = parsed_host_port.second;
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.collection = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
if (engine_args.size() >= 6)
options = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
configuration.options = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
}
return configuration;
}
void registerStorageMongoDB(StorageFactory & factory)
{
factory.registerStorage("MongoDB", [](const StorageFactory::Arguments & args)
{
auto configuration = StorageMongoDB::getConfiguration(args.engine_args, args.getLocalContext());
return StorageMongoDB::create(
args.table_id,
parsed_host_port.first,
parsed_host_port.second,
remote_database,
collection,
username,
password,
options,
configuration.host,
configuration.port,
configuration.database,
configuration.collection,
configuration.username,
configuration.password,
configuration.options,
args.columns,
args.constraints,
args.comment);

View File

@ -3,6 +3,7 @@
#include <common/shared_ptr_helper.h>
#include <Storages/IStorage.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <Poco/MongoDB/Connection.h>
@ -42,6 +43,8 @@ public:
size_t max_block_size,
unsigned num_streams) override;
static StorageMongoDBConfiguration getConfiguration(ASTs engine_args, ContextPtr context);
private:
void connectIfNotConnected();

View File

@ -234,68 +234,91 @@ SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMeta
local_context->getSettingsRef().mysql_max_rows_to_insert);
}
void registerStorageMySQL(StorageFactory & factory)
{
factory.registerStorage("MySQL", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
StorageMySQLConfiguration StorageMySQL::getConfiguration(ASTs engine_args, ContextPtr context_)
{
StorageMySQLConfiguration configuration;
if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context_))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
if (arg_name == "replace_query")
configuration.replace_query = arg_value.safeGet<bool>();
else if (arg_name == "on_duplicate_clause")
configuration.on_duplicate_clause = arg_value.safeGet<String>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unexpected key-value argument."
"Got: {}, but expected one of:"
"host, port, username, password, database, table, replace_query, on_duplicate_clause.", arg_name);
}
}
else
{
if (engine_args.size() < 5 || engine_args.size() > 7)
throw Exception(
"Storage MySQL requires 5-7 parameters: MySQL('host:port' (or 'addresses_pattern'), database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context_);
/// 3306 is the default MySQL port.
const String & host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
const String & username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = args.getContext()->getSettingsRef().glob_expansion_max_elements;
const auto & host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = context_->getSettingsRef().glob_expansion_max_elements;
/// TODO: move some arguments from the arguments to the SETTINGS.
MySQLSettings mysql_settings;
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
if (engine_args.size() >= 6)
configuration.replace_query = engine_args[5]->as<ASTLiteral &>().value.safeGet<UInt64>();
if (engine_args.size() == 7)
configuration.on_duplicate_clause = engine_args[6]->as<ASTLiteral &>().value.safeGet<String>();
}
if (configuration.replace_query && !configuration.on_duplicate_clause.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Only one of 'replace_query' and 'on_duplicate_clause' can be specified, or none of them");
return configuration;
}
void registerStorageMySQL(StorageFactory & factory)
{
factory.registerStorage("MySQL", [](const StorageFactory::Arguments & args)
{
auto configuration = StorageMySQL::getConfiguration(args.engine_args, args.getLocalContext());
MySQLSettings mysql_settings; /// TODO: move some arguments from the arguments to the SETTINGS.
if (args.storage_def->settings)
{
mysql_settings.loadFromQuery(*args.storage_def);
}
if (!mysql_settings.connection_pool_size)
throw Exception("connection_pool_size cannot be zero.", ErrorCodes::BAD_ARGUMENTS);
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
mysqlxx::PoolWithFailover pool(
remote_database,
addresses,
username,
password,
configuration.database, configuration.addresses,
configuration.username, configuration.password,
MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
mysql_settings.connection_pool_size,
mysql_settings.connection_max_tries,
mysql_settings.connection_wait_timeout);
bool replace_query = false;
std::string on_duplicate_clause;
if (engine_args.size() >= 6)
replace_query = engine_args[5]->as<ASTLiteral &>().value.safeGet<UInt64>();
if (engine_args.size() == 7)
on_duplicate_clause = engine_args[6]->as<ASTLiteral &>().value.safeGet<String>();
if (replace_query && !on_duplicate_clause.empty())
throw Exception(
"Only one of 'replace_query' and 'on_duplicate_clause' can be specified, or none of them",
ErrorCodes::BAD_ARGUMENTS);
return StorageMySQL::create(
args.table_id,
std::move(pool),
remote_database,
remote_table,
replace_query,
on_duplicate_clause,
configuration.database,
configuration.table,
configuration.replace_query,
configuration.on_duplicate_clause,
args.columns,
args.constraints,
args.comment,

View File

@ -10,6 +10,7 @@
#include <Storages/IStorage.h>
#include <Storages/MySQL/MySQLSettings.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <mysqlxx/PoolWithFailover.h>
@ -50,6 +51,8 @@ public:
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
static StorageMySQLConfiguration getConfiguration(ASTs engine_args, ContextPtr context_);
private:
friend class StorageMySQLSink;

View File

@ -3,43 +3,51 @@
#if USE_LIBPQXX
#include <DataStreams/PostgreSQLSource.h>
#include <Storages/StorageFactory.h>
#include <Storages/transformQueryForExternalDatabase.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
#include <Common/parseAddress.h>
#include <Common/assert_cast.h>
#include <Common/parseRemoteDescription.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
#include <Core/Settings.h>
#include <Common/parseAddress.h>
#include <Common/assert_cast.h>
#include <Parsers/ASTLiteral.h>
#include <Columns/ColumnNullable.h>
#include <Formats/FormatFactory.h>
#include <Formats/FormatSettings.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Common/parseRemoteDescription.h>
#include <Processors/Pipe.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <IO/WriteHelpers.h>
#include <Parsers/getInsertQuery.h>
#include <IO/Operators.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
#include <Parsers/getInsertQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <Storages/StorageFactory.h>
#include <Storages/transformQueryForExternalDatabase.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
StoragePostgreSQL::StoragePostgreSQL(
@ -376,53 +384,80 @@ SinkToStoragePtr StoragePostgreSQL::write(
}
StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_args, ContextPtr context)
{
StoragePostgreSQLConfiguration configuration;
if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
if (arg_name == "on_conflict")
configuration.on_conflict = arg_value.safeGet<String>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unexpected key-value argument."
"Got: {}, but expected one of:"
"host, port, username, password, database, table, schema, on_conflict.", arg_name);
}
}
else
{
if (engine_args.size() < 5 || engine_args.size() > 7)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage PostgreSQL requires from 5 to 7 parameters: "
"PostgreSQL('host:port', 'database', 'table', 'username', 'password' [, 'schema', 'ON CONFLICT ...']. Got: {}",
engine_args.size());
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
const auto & host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
if (configuration.addresses.size() == 1)
{
configuration.host = configuration.addresses[0].first;
configuration.port = configuration.addresses[0].second;
}
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
if (engine_args.size() >= 6)
configuration.schema = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
if (engine_args.size() >= 7)
configuration.on_conflict = engine_args[6]->as<ASTLiteral &>().value.safeGet<String>();
}
return configuration;
}
void registerStoragePostgreSQL(StorageFactory & factory)
{
factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
if (engine_args.size() < 5 || engine_args.size() > 7)
throw Exception("Storage PostgreSQL requires from 5 to 7 parameters: "
"PostgreSQL('host:port', 'database', 'table', 'username', 'password' [, 'schema', 'ON CONFLICT ...']",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
auto host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
/// Split into replicas if needed.
size_t max_addresses = args.getContext()->getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
const String & username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
const String & password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
String remote_table_schema, on_conflict;
if (engine_args.size() >= 6)
remote_table_schema = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
if (engine_args.size() >= 7)
on_conflict = engine_args[6]->as<ASTLiteral &>().value.safeGet<String>();
auto pool = std::make_shared<postgres::PoolWithFailover>(
remote_database,
addresses,
username,
password,
auto configuration = StoragePostgreSQL::getConfiguration(args.engine_args, args.getLocalContext());
auto pool = std::make_shared<postgres::PoolWithFailover>(configuration,
args.getContext()->getSettingsRef().postgresql_connection_pool_size,
args.getContext()->getSettingsRef().postgresql_connection_pool_wait_timeout);
return StoragePostgreSQL::create(
args.table_id,
std::move(pool),
remote_table,
configuration.table,
args.columns,
args.constraints,
args.comment,
remote_table_schema,
on_conflict);
configuration.schema,
configuration.on_conflict);
},
{
.source_access_type = AccessType::POSTGRES,

View File

@ -10,12 +10,12 @@
#include <Storages/IStorage.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Core/PostgreSQL/PoolWithFailover.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace DB
{
class StoragePostgreSQL final : public shared_ptr_helper<StoragePostgreSQL>, public IStorage
{
friend struct shared_ptr_helper<StoragePostgreSQL>;
@ -43,6 +43,8 @@ public:
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
static StoragePostgreSQLConfiguration getConfiguration(ASTs engine_args, ContextPtr context);
private:
friend class PostgreSQLBlockOutputStream;

View File

@ -732,20 +732,70 @@ void StorageS3::updateClientAndAuthSettings(ContextPtr ctx, StorageS3::ClientAut
upd.auth_settings = std::move(settings);
}
void registerStorageS3Impl(const String & name, StorageFactory & factory)
{
factory.registerStorage(name, [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPtr local_context)
{
StorageS3Configuration configuration;
if (auto named_collection = getURLBasedDataSourceConfiguration(engine_args, local_context))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
if (arg_name == "access_key_id")
configuration.access_key_id = arg_value.safeGet<String>();
else if (arg_name == "secret_access_key")
configuration.secret_access_key = arg_value.safeGet<String>();
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
arg_name);
}
}
else
{
if (engine_args.size() < 2 || engine_args.size() > 5)
throw Exception(
"Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context);
configuration.url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
if (engine_args.size() >= 4)
{
configuration.access_key_id = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.secret_access_key = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
if (engine_args.size() == 3 || engine_args.size() == 5)
{
configuration.compression_method = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
configuration.format = engine_args[engine_args.size() - 2]->as<ASTLiteral &>().value.safeGet<String>();
}
else
{
configuration.compression_method = "auto";
configuration.format = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
}
}
return configuration;
}
void registerStorageS3Impl(const String & name, StorageFactory & factory)
{
factory.registerStorage(name, [](const StorageFactory::Arguments & args)
{
auto & engine_args = args.engine_args;
if (engine_args.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments");
auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext());
// Use format settings from global server context + settings from
// the SETTINGS clause of the create query. Settings from current
// session and user are ignored.
@ -760,9 +810,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
for (const auto & change : changes)
{
if (user_format_settings.has(change.name))
{
user_format_settings.set(change.name, change.value);
}
}
// Apply changes from SETTINGS clause, with validation.
@ -774,42 +822,18 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
format_settings = getFormatSettings(args.getContext());
}
String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
Poco::URI uri (url);
S3::URI s3_uri (uri);
String access_key_id;
String secret_access_key;
if (engine_args.size() >= 4)
{
access_key_id = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
secret_access_key = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
UInt64 max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries;
UInt64 min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size;
UInt64 max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size;
UInt64 max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections;
String compression_method;
String format_name;
if (engine_args.size() == 3 || engine_args.size() == 5)
{
compression_method = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
format_name = engine_args[engine_args.size() - 2]->as<ASTLiteral &>().value.safeGet<String>();
}
else
{
compression_method = "auto";
format_name = engine_args.back()->as<ASTLiteral &>().value.safeGet<String>();
}
S3::URI s3_uri(Poco::URI(configuration.url));
auto max_single_read_retries = args.getLocalContext()->getSettingsRef().s3_max_single_read_retries;
auto min_upload_part_size = args.getLocalContext()->getSettingsRef().s3_min_upload_part_size;
auto max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size;
auto max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections;
return StorageS3::create(
s3_uri,
access_key_id,
secret_access_key,
configuration.access_key_id,
configuration.secret_access_key,
args.table_id,
format_name,
configuration.format,
max_single_read_retries,
min_upload_part_size,
max_single_part_upload_size,
@ -819,7 +843,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
args.comment,
args.getContext(),
format_settings,
compression_method);
configuration.compression_method);
},
{
.supports_settings = true,

View File

@ -18,6 +18,7 @@
#include <IO/S3Common.h>
#include <IO/CompressionMethod.h>
#include <Interpreters/Context.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace Aws::S3
{
@ -141,6 +142,8 @@ public:
bool supportsPartitionBy() const override;
static StorageS3Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context);
private:
friend class StorageS3Cluster;

View File

@ -6,7 +6,6 @@
#include <Parsers/ASTLiteral.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/WriteBufferFromHTTP.h>
#include <IO/WriteHelpers.h>
#include <IO/ConnectionTimeouts.h>
@ -32,8 +31,10 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NETWORK_ERROR;
extern const int BAD_ARGUMENTS;
}
IStorageURLBase::IStorageURLBase(
const Poco::URI & uri_,
ContextPtr /*context_*/,
@ -43,8 +44,9 @@ IStorageURLBase::IStorageURLBase(
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
const String & compression_method_)
: IStorage(table_id_), uri(uri_), compression_method(compression_method_), format_name(format_name_), format_settings(format_settings_)
const String & compression_method_,
const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_)
: IStorage(table_id_), uri(uri_), compression_method(compression_method_), format_name(format_name_), format_settings(format_settings_), headers(headers_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
@ -69,10 +71,14 @@ namespace
const ColumnsDescription & columns,
UInt64 max_block_size,
const ConnectionTimeouts & timeouts,
const CompressionMethod compression_method)
const CompressionMethod compression_method,
const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {})
: SourceWithProgress(sample_block), name(std::move(name_))
{
ReadWriteBufferFromHTTP::HTTPHeaderEntries header;
ReadWriteBufferFromHTTP::HTTPHeaderEntries headers;
for (const auto & header : headers_)
headers.emplace_back(header);
// Propagate OpenTelemetry trace context, if any, downstream.
if (CurrentThread::isInitialized())
@ -80,12 +86,12 @@ namespace
const auto & thread_trace_context = CurrentThread::get().thread_trace_context;
if (thread_trace_context.trace_id != UUID())
{
header.emplace_back("traceparent",
headers.emplace_back("traceparent",
thread_trace_context.composeTraceparentHeader());
if (!thread_trace_context.tracestate.empty())
{
header.emplace_back("tracestate",
headers.emplace_back("tracestate",
thread_trace_context.tracestate);
}
}
@ -100,7 +106,7 @@ namespace
context->getSettingsRef().max_http_get_redirects,
Poco::Net::HTTPBasicCredentials{},
DBMS_DEFAULT_BUFFER_SIZE,
header,
headers,
context->getRemoteHostFilter()),
compression_method);
@ -237,7 +243,8 @@ Pipe IStorageURLBase::read(
metadata_snapshot->getColumns(),
max_block_size,
ConnectionTimeouts::getHTTPTimeouts(local_context),
chooseCompressionMethod(request_uri.getPath(), compression_method)));
chooseCompressionMethod(request_uri.getPath(), compression_method),
headers));
}
@ -312,8 +319,9 @@ StorageURL::StorageURL(
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
const String & compression_method_)
: IStorageURLBase(uri_, context_, table_id_, format_name_, format_settings_, columns_, constraints_, comment, compression_method_)
const String & compression_method_,
const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_)
: IStorageURLBase(uri_, context_, table_id_, format_name_, format_settings_, columns_, constraints_, comment, compression_method_, headers_)
{
context_->getRemoteHostFilter().checkURL(uri);
}
@ -375,45 +383,73 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum
return format_settings;
}
URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, ContextPtr local_context)
{
URLBasedDataSourceConfiguration configuration;
if (auto named_collection = getURLBasedDataSourceConfiguration(args, local_context))
{
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
if (!storage_specific_args.empty())
{
String illegal_args;
for (const auto & arg : storage_specific_args)
{
if (!illegal_args.empty())
illegal_args += ", ";
illegal_args += arg.first;
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguments {} for table function URL", illegal_args);
}
}
else
{
if (args.size() != 2 && args.size() != 3)
throw Exception(
"Storage URL requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, local_context);
configuration.url = args[0]->as<ASTLiteral &>().value.safeGet<String>();
configuration.format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
if (args.size() == 3)
configuration.compression_method = args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
return configuration;
}
void registerStorageURL(StorageFactory & factory)
{
factory.registerStorage("URL", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
if (engine_args.size() != 2 && engine_args.size() != 3)
throw Exception(
"Storage URL requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.getLocalContext());
const String & url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
Poco::URI uri(url);
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.getLocalContext());
const String & format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
String compression_method = "auto";
if (engine_args.size() == 3)
{
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext());
compression_method = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
auto configuration = StorageURL::getConfiguration(engine_args, args.getLocalContext());
auto format_settings = StorageURL::getFormatSettingsFromArgs(args);
Poco::URI uri(configuration.url);
ReadWriteBufferFromHTTP::HTTPHeaderEntries headers;
for (const auto & [header, value] : configuration.headers)
{
auto value_literal = value.safeGet<String>();
headers.emplace_back(std::make_pair(header, value_literal));
}
return StorageURL::create(
uri,
args.table_id,
format_name,
configuration.format,
format_settings,
args.columns,
args.constraints,
args.comment,
args.getContext(),
compression_method);
configuration.compression_method,
headers);
},
{
.supports_settings = true,

View File

@ -6,7 +6,9 @@
#include <Processors/Sinks/SinkToStorage.h>
#include <Formats/FormatSettings.h>
#include <IO/CompressionMethod.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Storages/StorageFactory.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace DB
@ -44,7 +46,8 @@ protected:
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
const String & compression_method_);
const String & compression_method_,
const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {});
Poco::URI uri;
String compression_method;
@ -54,6 +57,7 @@ protected:
// For `url` table function, we use settings from current query context.
// In this case, format_settings is not set.
std::optional<FormatSettings> format_settings;
ReadWriteBufferFromHTTP::HTTPHeaderEntries headers;
virtual std::string getReadMethod() const;
@ -113,7 +117,8 @@ public:
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
const String & compression_method_);
const String & compression_method_,
const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {});
String getName() const override
{
@ -126,6 +131,8 @@ public:
}
static FormatSettings getFormatSettingsFromArgs(const StorageFactory::Arguments & args);
static URLBasedDataSourceConfiguration getConfiguration(ASTs & args, ContextPtr context);
};
@ -152,6 +159,13 @@ public:
size_t max_block_size,
unsigned num_streams) override;
struct Configuration
{
String url;
String compression_method = "auto";
std::vector<std::pair<String, String>> headers;
};
private:
std::vector<Poco::URI> uri_options;
};

View File

@ -158,8 +158,8 @@ void StorageView::read(
{
throw DB::Exception(ErrorCodes::INCORRECT_QUERY,
"Query from view {} returned Nullable column having not Nullable type in structure. "
"If query from view has JOIN, it may be cause by different values of 'json_use_nulls' setting. "
"You may explicitly specify 'json_use_nulls' in 'CREATE VIEW' query to avoid this error",
"If query from view has JOIN, it may be cause by different values of 'join_use_nulls' setting. "
"You may explicitly specify 'join_use_nulls' in 'CREATE VIEW' query to avoid this error",
getStorageID().getFullTableName());
}

View File

@ -1,23 +1,14 @@
#include <algorithm>
#include <filesystem>
#include "StorageSystemDDLWorkerQueue.h"
#include <Columns/ColumnArray.h>
#include <Storages/System/StorageSystemDDLWorkerQueue.h>
#include <Interpreters/DDLTask.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeMap.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
#include <Storages/SelectQueryInfo.h>
#include <Interpreters/ZooKeeperLog.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
@ -25,207 +16,334 @@
namespace fs = std::filesystem;
enum Status
{
ACTIVE,
FINISHED,
UNKNOWN,
ERRORED
};
namespace DB
{
std::vector<std::pair<String, Int8>> getStatusEnumsAndValues()
enum class Status
{
INACTIVE,
ACTIVE,
FINISHED,
REMOVING,
UNKNOWN,
};
using GetResponseFuture = std::future<Coordination::GetResponse>;
using ListResponseFuture = std::future<Coordination::ListResponse>;
using GetResponseFutures = std::vector<GetResponseFuture>;
using ListResponseFutures = std::vector<ListResponseFuture>;
static std::vector<std::pair<String, Int8>> getStatusEnumsAndValues()
{
return std::vector<std::pair<String, Int8>>{
{"Active", static_cast<Int8>(Status::ACTIVE)},
{"Finished", static_cast<Int8>(Status::FINISHED)},
{"Unknown", static_cast<Int8>(Status::UNKNOWN)},
{"Errored", static_cast<Int8>(Status::ERRORED)},
{"Inactive", static_cast<Int8>(Status::INACTIVE)},
{"Active", static_cast<Int8>(Status::ACTIVE)},
{"Finished", static_cast<Int8>(Status::FINISHED)},
{"Removing", static_cast<Int8>(Status::REMOVING)},
{"Unknown", static_cast<Int8>(Status::UNKNOWN)},
};
}
std::vector<std::pair<String, Int8>> getZooKeeperErrorEnumsAndValues()
{
return std::vector<std::pair<String, Int8>>{
{"ZOK", static_cast<Int8>(Coordination::Error::ZOK)},
{"ZSYSTEMERROR", static_cast<Int8>(Coordination::Error::ZSYSTEMERROR)},
{"ZRUNTIMEINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZRUNTIMEINCONSISTENCY)},
{"ZDATAINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZDATAINCONSISTENCY)},
{"ZCONNECTIONLOSS", static_cast<Int8>(Coordination::Error::ZCONNECTIONLOSS)},
{"ZMARSHALLINGERROR", static_cast<Int8>(Coordination::Error::ZMARSHALLINGERROR)},
{"ZUNIMPLEMENTED", static_cast<Int8>(Coordination::Error::ZUNIMPLEMENTED)},
{"ZOPERATIONTIMEOUT", static_cast<Int8>(Coordination::Error::ZOPERATIONTIMEOUT)},
{"ZBADARGUMENTS", static_cast<Int8>(Coordination::Error::ZBADARGUMENTS)},
{"ZINVALIDSTATE", static_cast<Int8>(Coordination::Error::ZINVALIDSTATE)},
{"ZAPIERROR", static_cast<Int8>(Coordination::Error::ZAPIERROR)},
{"ZNONODE", static_cast<Int8>(Coordination::Error::ZNONODE)},
{"ZNOAUTH", static_cast<Int8>(Coordination::Error::ZNOAUTH)},
{"ZBADVERSION", static_cast<Int8>(Coordination::Error::ZBADVERSION)},
{"ZNOCHILDRENFOREPHEMERALS", static_cast<Int8>(Coordination::Error::ZNOCHILDRENFOREPHEMERALS)},
{"ZNODEEXISTS", static_cast<Int8>(Coordination::Error::ZNODEEXISTS)},
{"ZNOTEMPTY", static_cast<Int8>(Coordination::Error::ZNOTEMPTY)},
{"ZSESSIONEXPIRED", static_cast<Int8>(Coordination::Error::ZSESSIONEXPIRED)},
{"ZINVALIDCALLBACK", static_cast<Int8>(Coordination::Error::ZINVALIDCALLBACK)},
{"ZINVALIDACL", static_cast<Int8>(Coordination::Error::ZINVALIDACL)},
{"ZAUTHFAILED", static_cast<Int8>(Coordination::Error::ZAUTHFAILED)},
{"ZCLOSING", static_cast<Int8>(Coordination::Error::ZCLOSING)},
{"ZNOTHING", static_cast<Int8>(Coordination::Error::ZNOTHING)},
{"ZSESSIONMOVED", static_cast<Int8>(Coordination::Error::ZSESSIONMOVED)},
};
}
NamesAndTypesList StorageSystemDDLWorkerQueue::getNamesAndTypes()
{
return {
{"entry", std::make_shared<DataTypeString>()},
{"host_name", std::make_shared<DataTypeString>()},
{"host_address", std::make_shared<DataTypeString>()},
{"port", std::make_shared<DataTypeUInt16>()},
{"status", std::make_shared<DataTypeEnum8>(getStatusEnumsAndValues())},
{"cluster", std::make_shared<DataTypeString>()},
{"query", std::make_shared<DataTypeString>()},
{"initiator", std::make_shared<DataTypeString>()},
{"query_start_time", std::make_shared<DataTypeDateTime>()},
{"query_finish_time", std::make_shared<DataTypeDateTime>()},
{"query_duration_ms", std::make_shared<DataTypeUInt64>()},
{"exception_code", std::make_shared<DataTypeEnum8>(getZooKeeperErrorEnumsAndValues())},
{"entry", std::make_shared<DataTypeString>()},
{"entry_version", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt8>())},
{"initiator_host", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"initiator_port", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt16>())},
{"cluster", std::make_shared<DataTypeString>()},
{"query", std::make_shared<DataTypeString>()},
{"settings", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeString>())},
{"query_create_time", std::make_shared<DataTypeDateTime>()},
{"host", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"port", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt16>())},
{"status", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeEnum8>(getStatusEnumsAndValues()))},
{"exception_code", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt16>())},
{"exception_text", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"query_finish_time", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeDateTime>())},
{"query_duration_ms", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
};
}
static String clusterNameFromDDLQuery(ContextPtr context, const DDLLogEntry & entry)
static String clusterNameFromDDLQuery(ContextPtr context, const DDLTask & task)
{
const char * begin = entry.query.data();
const char * end = begin + entry.query.size();
ASTPtr query;
ASTQueryWithOnCluster * query_on_cluster;
const char * begin = task.entry.query.data();
const char * end = begin + task.entry.query.size();
String cluster_name;
ParserQuery parser_query(end);
String description;
query = parseQuery(parser_query, begin, end, description, 0, context->getSettingsRef().max_parser_depth);
if (query && (query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(query.get())))
String description = fmt::format("from {}", task.entry_path);
ASTPtr query = parseQuery(parser_query, begin, end, description,
context->getSettingsRef().max_query_size, context->getSettingsRef().max_parser_depth);
if (const auto * query_on_cluster = dynamic_cast<const ASTQueryWithOnCluster *>(query.get()))
cluster_name = query_on_cluster->cluster;
return cluster_name;
}
static void fillCommonColumns(MutableColumns & res_columns, size_t & col, const DDLTask & task, const String & cluster_name, UInt64 query_create_time_ms)
{
/// entry
res_columns[col++]->insert(task.entry_name);
/// entry_version
res_columns[col++]->insert(task.entry.version);
if (task.entry.initiator.empty())
{
/// initiator_host
res_columns[col++]->insert(Field{});
/// initiator_port
res_columns[col++]->insert(Field{});
}
else
{
HostID initiator = HostID::fromString(task.entry.initiator);
/// initiator_host
res_columns[col++]->insert(initiator.host_name);
/// initiator_port
res_columns[col++]->insert(initiator.port);
}
/// cluster
res_columns[col++]->insert(cluster_name);
/// query
res_columns[col++]->insert(task.entry.query);
Map settings_map;
if (task.entry.settings)
{
for (const auto & change : *task.entry.settings)
{
Tuple pair;
pair.push_back(change.name);
pair.push_back(toString(change.value));
settings_map.push_back(std::move(pair));
}
}
/// settings
res_columns[col++]->insert(settings_map);
res_columns[col++]->insert(static_cast<UInt64>(query_create_time_ms / 1000));
}
static void repeatValuesInCommonColumns(MutableColumns & res_columns, size_t num_filled_columns)
{
if (res_columns[num_filled_columns - 1]->size() == res_columns[num_filled_columns]->size() + 1)
{
/// Common columns are already filled
return;
}
/// Copy values from previous row
assert(res_columns[num_filled_columns - 1]->size() == res_columns[num_filled_columns]->size());
for (size_t filled_col = 0; filled_col < num_filled_columns; ++filled_col)
res_columns[filled_col]->insert((*res_columns[filled_col])[res_columns[filled_col]->size() - 1]);
}
static void fillHostnameColumns(MutableColumns & res_columns, size_t & col, const HostID & host_id)
{
/// NOTE host_id.host_name can be a domain name or an IP address
/// We could try to resolve domain name or reverse resolve an address and add two separate columns,
/// but seems like it's not really needed, so we show host_id.host_name as is.
/// host
res_columns[col++]->insert(host_id.host_name);
/// port
res_columns[col++]->insert(host_id.port);
}
static void fillStatusColumnsWithNulls(MutableColumns & res_columns, size_t & col, Status status)
{
/// status
res_columns[col++]->insert(static_cast<Int8>(status));
/// exception_code
res_columns[col++]->insert(Field{});
/// exception_text
res_columns[col++]->insert(Field{});
/// query_finish_time
res_columns[col++]->insert(Field{});
/// query_duration_ms
res_columns[col++]->insert(Field{});
}
static void fillStatusColumns(MutableColumns & res_columns, size_t & col,
GetResponseFuture & finished_data_future,
UInt64 query_create_time_ms)
{
auto maybe_finished_status = finished_data_future.get();
if (maybe_finished_status.error == Coordination::Error::ZNONODE)
return fillStatusColumnsWithNulls(res_columns, col, Status::REMOVING);
/// asyncTryGet should throw on other error codes
assert(maybe_finished_status.error == Coordination::Error::ZOK);
/// status
res_columns[col++]->insert(static_cast<Int8>(Status::FINISHED));
auto execution_status = ExecutionStatus::fromText(maybe_finished_status.data);
/// exception_code
res_columns[col++]->insert(execution_status.code);
/// exception_text
res_columns[col++]->insert(execution_status.message);
UInt64 query_finish_time_ms = maybe_finished_status.stat.ctime;
/// query_finish_time
res_columns[col++]->insert(static_cast<UInt64>(query_finish_time_ms / 1000));
/// query_duration_ms
res_columns[col++]->insert(static_cast<UInt64>(query_finish_time_ms - query_create_time_ms));
}
void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper();
Coordination::Error zk_exception_code = Coordination::Error::ZOK;
String ddl_zookeeper_path = config.getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
String ddl_query_path;
fs::path ddl_zookeeper_path = context->getConfigRef().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
// this is equivalent to query zookeeper at the `ddl_zookeeper_path`
/* [zk: localhost:2181(CONNECTED) 51] ls /clickhouse/task_queue/ddl
[query-0000000000, query-0000000001, query-0000000002, query-0000000003, query-0000000004]
*/
Strings ddl_task_paths = zookeeper->getChildren(ddl_zookeeper_path);
zkutil::Strings queries;
GetResponseFutures ddl_task_futures;
ListResponseFutures active_nodes_futures;
ListResponseFutures finished_nodes_futures;
Coordination::Error code = zookeeper->tryGetChildren(ddl_zookeeper_path, queries);
// if there is an error here, just register the code.
// the queries will be empty and so there will be nothing to fill the table
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
zk_exception_code = code;
const auto clusters = context->getClusters();
for (const auto & name_and_cluster : clusters->getContainer())
for (const auto & task_path : ddl_task_paths)
{
const ClusterPtr & cluster = name_and_cluster.second;
const auto & shards_info = cluster->getShardsInfo();
const auto & addresses_with_failover = cluster->getShardsAddresses();
for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index)
ddl_task_futures.push_back(zookeeper->asyncTryGet(ddl_zookeeper_path / task_path));
/// List status dirs. Active host may become finished, so we list active first.
active_nodes_futures.push_back(zookeeper->asyncTryGetChildrenNoThrow(ddl_zookeeper_path / task_path / "active"));
finished_nodes_futures.push_back(zookeeper->asyncTryGetChildrenNoThrow(ddl_zookeeper_path / task_path / "finished"));
}
for (size_t i = 0; i < ddl_task_paths.size(); ++i)
{
auto maybe_task = ddl_task_futures[i].get();
if (maybe_task.error != Coordination::Error::ZOK)
{
const auto & shard_addresses = addresses_with_failover[shard_index];
const auto & shard_info = shards_info[shard_index];
const auto pool_status = shard_info.pool->getStatus();
for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index)
/// Task is removed
assert(maybe_task.error == Coordination::Error::ZNONODE);
continue;
}
DDLTask task{ddl_task_paths[i], ddl_zookeeper_path / ddl_task_paths[i]};
try
{
task.entry.parse(maybe_task.data);
}
catch (Exception & e)
{
e.addMessage("On parsing DDL entry {}: {}", task.entry_path, maybe_task.data);
throw;
}
String cluster_name = clusterNameFromDDLQuery(context, task);
UInt64 query_create_time_ms = maybe_task.stat.ctime;
size_t col = 0;
fillCommonColumns(res_columns, col, task, cluster_name, query_create_time_ms);
/// At first we process finished nodes, to avoid duplication if some host was active
/// and suddenly become finished during status dirs listing.
/// Then we process active (but not finished) hosts.
/// And then we process the rest hosts from task.entry.hosts list.
/// NOTE: It's not guaranteed that task.entry.hosts contains all host ids from status dirs.
std::unordered_set<String> processed_hosts;
/// Race condition with DDLWorker::cleanupQueue(...) is possible.
/// We may get incorrect list of finished nodes if task is currently removing.
/// To avoid showing INACTIVE status for hosts that have actually executed query,
/// we will detect if someone is removing task and show special REMOVING status.
/// Also we should distinguish it from another case when status dirs are not created yet (extremely rare case).
bool is_removing_task = false;
auto maybe_finished_hosts = finished_nodes_futures[i].get();
if (maybe_finished_hosts.error == Coordination::Error::ZOK)
{
GetResponseFutures finished_status_futures;
for (const auto & host_id_str : maybe_finished_hosts.names)
finished_status_futures.push_back(zookeeper->asyncTryGet(fs::path(task.entry_path) / "finished" / host_id_str));
for (size_t host_idx = 0; host_idx < maybe_finished_hosts.names.size(); ++host_idx)
{
/* Dir contents of every query will be similar to
[zk: localhost:2181(CONNECTED) 53] ls /clickhouse/task_queue/ddl/query-0000000004
[active, finished]
*/
std::vector<std::future<Coordination::GetResponse>> futures;
futures.reserve(queries.size());
for (const String & q : queries)
{
futures.push_back(zookeeper->asyncTryGet(fs::path(ddl_zookeeper_path) / q));
}
for (size_t query_id = 0; query_id < queries.size(); query_id++)
{
Int64 query_finish_time = 0;
size_t i = 0;
res_columns[i++]->insert(queries[query_id]); // entry
const auto & address = shard_addresses[replica_index];
res_columns[i++]->insert(address.host_name);
auto resolved = address.getResolvedAddress();
res_columns[i++]->insert(resolved ? resolved->host().toString() : String()); // host_address
res_columns[i++]->insert(address.port);
ddl_query_path = fs::path(ddl_zookeeper_path) / queries[query_id];
zkutil::Strings active_nodes;
zkutil::Strings finished_nodes;
code = zookeeper->tryGetChildren(fs::path(ddl_query_path) / "active", active_nodes);
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
zk_exception_code = code;
code = zookeeper->tryGetChildren(fs::path(ddl_query_path) / "finished", finished_nodes);
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
zk_exception_code = code;
/* status:
* active: If the hostname:port entry is present under active path.
* finished: If the hostname:port entry is present under the finished path.
* errored: If the hostname:port entry is present under the finished path but the error count is not 0.
* unknown: If the above cases don't hold true, then status is unknown.
*/
if (std::find(active_nodes.begin(), active_nodes.end(), address.toString()) != active_nodes.end())
{
res_columns[i++]->insert(static_cast<Int8>(Status::ACTIVE));
}
else if (std::find(finished_nodes.begin(), finished_nodes.end(), address.toString()) != finished_nodes.end())
{
if (pool_status[replica_index].error_count != 0)
{
res_columns[i++]->insert(static_cast<Int8>(Status::ERRORED));
}
else
{
res_columns[i++]->insert(static_cast<Int8>(Status::FINISHED));
}
// regardless of the status finished or errored, the node host_name:port entry was found under the /finished path
// & should be able to get the contents of the znode at /finished path.
auto res_fn = zookeeper->asyncTryGet(fs::path(ddl_query_path) / "finished");
auto stat_fn = res_fn.get().stat;
query_finish_time = stat_fn.mtime;
}
else
{
res_columns[i++]->insert(static_cast<Int8>(Status::UNKNOWN));
}
Coordination::GetResponse res;
res = futures[query_id].get();
auto query_start_time = res.stat.mtime;
DDLLogEntry entry;
entry.parse(res.data);
String cluster_name = clusterNameFromDDLQuery(context, entry);
res_columns[i++]->insert(cluster_name);
res_columns[i++]->insert(entry.query);
res_columns[i++]->insert(entry.initiator);
res_columns[i++]->insert(UInt64(query_start_time / 1000));
res_columns[i++]->insert(UInt64(query_finish_time / 1000));
res_columns[i++]->insert(UInt64(query_finish_time - query_start_time));
res_columns[i++]->insert(static_cast<Int8>(zk_exception_code));
}
const auto & host_id_str = maybe_finished_hosts.names[host_idx];
HostID host_id = HostID::fromString(host_id_str);
repeatValuesInCommonColumns(res_columns, col);
size_t rest_col = col;
fillHostnameColumns(res_columns, rest_col, host_id);
fillStatusColumns(res_columns, rest_col, finished_status_futures[host_idx], query_create_time_ms);
processed_hosts.insert(host_id_str);
}
}
else if (maybe_finished_hosts.error == Coordination::Error::ZNONODE)
{
/// Rare case: Either status dirs are not created yet or already removed.
/// We can distinguish it by checking if task node exists, because "query-xxx" and "query-xxx/finished"
/// are removed in single multi-request
is_removing_task = !zookeeper->exists(task.entry_path);
}
else
{
throw Coordination::Exception(maybe_finished_hosts.error, fs::path(task.entry_path) / "finished");
}
/// Process active nodes
auto maybe_active_hosts = active_nodes_futures[i].get();
if (maybe_active_hosts.error == Coordination::Error::ZOK)
{
for (const auto & host_id_str : maybe_active_hosts.names)
{
if (processed_hosts.contains(host_id_str))
continue;
HostID host_id = HostID::fromString(host_id_str);
repeatValuesInCommonColumns(res_columns, col);
size_t rest_col = col;
fillHostnameColumns(res_columns, rest_col, host_id);
fillStatusColumnsWithNulls(res_columns, rest_col, Status::ACTIVE);
processed_hosts.insert(host_id_str);
}
}
else if (maybe_active_hosts.error == Coordination::Error::ZNONODE)
{
/// Rare case: Either status dirs are not created yet or task is currently removing.
/// When removing a task, at first we remove "query-xxx/active" (not recursively),
/// then recursively remove everything except "query-xxx/finished"
/// and then remove "query-xxx" and "query-xxx/finished".
is_removing_task = is_removing_task ||
(zookeeper->exists(fs::path(task.entry_path) / "finished") && !zookeeper->exists(fs::path(task.entry_path) / "active")) ||
!zookeeper->exists(task.entry_path);
}
else
{
throw Coordination::Exception(maybe_active_hosts.error, fs::path(task.entry_path) / "active");
}
/// Process the rest hosts
for (const auto & host_id : task.entry.hosts)
{
if (processed_hosts.contains(host_id.toString()))
continue;
Status status = is_removing_task ? Status::REMOVING : Status::INACTIVE;
repeatValuesInCommonColumns(res_columns, col);
size_t rest_col = col;
fillHostnameColumns(res_columns, rest_col, host_id);
fillStatusColumnsWithNulls(res_columns, rest_col, status);
processed_hosts.insert(host_id.toString());
}
if (processed_hosts.empty())
{
/// We don't know any hosts, just fill the rest columns with nulls.
/// host
res_columns[col++]->insert(Field{});
/// port
res_columns[col++]->insert(Field{});
fillStatusColumnsWithNulls(res_columns, col, Status::UNKNOWN);
}
}
}
}

View File

@ -1,12 +1,8 @@
#pragma once
#include <Poco/Util/Application.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <Interpreters/DDLWorker.h>
#include <Storages/System/IStorageSystemOneBlock.h>
#include <common/shared_ptr_helper.h>
#include <future>
namespace DB
{
@ -19,10 +15,9 @@ class StorageSystemDDLWorkerQueue final : public shared_ptr_helper<StorageSystem
public IStorageSystemOneBlock<StorageSystemDDLWorkerQueue>
{
friend struct shared_ptr_helper<StorageSystemDDLWorkerQueue>;
Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config();
protected:
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override;
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override;
using IStorageSystemOneBlock::IStorageSystemOneBlock;

View File

@ -219,3 +219,33 @@ TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere)
"WHERE column > 2 AND (apply_id = 1 OR table2.num = 1) AND table2.attr != ''",
R"(SELECT "column", "apply_id" FROM "test"."table" WHERE ("column" > 2) AND ("apply_id" = 1))");
}
TEST(TransformQueryForExternalDatabase, NoStrict)
{
const State & state = State::instance();
check(state, 1,
"SELECT field FROM table WHERE field IN (SELECT attr FROM table2)",
R"(SELECT "field" FROM "test"."table")");
}
TEST(TransformQueryForExternalDatabase, Strict)
{
const State & state = State::instance();
state.context->setSetting("external_table_strict_query", true);
check(state, 1,
"SELECT field FROM table WHERE field = '1'",
R"(SELECT "field" FROM "test"."table" WHERE "field" = '1')");
check(state, 1,
"SELECT field FROM table WHERE field IN ('1', '2')",
R"(SELECT "field" FROM "test"."table" WHERE "field" IN ('1', '2'))");
check(state, 1,
"SELECT field FROM table WHERE field LIKE '%test%'",
R"(SELECT "field" FROM "test"."table" WHERE "field" LIKE '%test%')");
/// removeUnknownSubexpressionsFromWhere() takes place
EXPECT_THROW(check(state, 1, "SELECT field FROM table WHERE field IN (SELECT attr FROM table2)", ""), Exception);
/// !isCompatible() takes place
EXPECT_THROW(check(state, 1, "SELECT column FROM test.table WHERE left(column, 10) = RIGHT(column, 10) AND SUBSTRING(column FROM 1 FOR 2) = 'Hello'", ""), Exception);
}

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTExpressionList.h>
#include <Interpreters/TreeRewriter.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/Context.h>
#include <IO/WriteBufferFromString.h>
#include <Storages/transformQueryForExternalDatabase.h>
#include <Storages/MergeTree/KeyCondition.h>
@ -20,6 +21,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int INCORRECT_QUERY;
}
namespace
@ -248,6 +250,7 @@ String transformQueryForExternalDatabase(
{
auto clone_query = query_info.query->clone();
const Names used_columns = query_info.syntax_analyzer_result->requiredSourceColumns();
bool strict = context->getSettingsRef().external_table_strict_query;
auto select = std::make_shared<ASTSelectQuery>();
@ -275,6 +278,10 @@ String transformQueryForExternalDatabase(
{
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where));
}
else if (strict)
{
throw Exception("Query contains non-compatible expressions (and external_table_strict_query=true)", ErrorCodes::INCORRECT_QUERY);
}
else if (const auto * function = original_where->as<ASTFunction>())
{
if (function->name == "and")
@ -292,6 +299,10 @@ String transformQueryForExternalDatabase(
}
}
}
else if (strict && original_where)
{
throw Exception("Query contains non-compatible expressions (and external_table_strict_query=true)", ErrorCodes::INCORRECT_QUERY);
}
ASTPtr select_ptr = select;
dropAliases(select_ptr);

View File

@ -22,6 +22,9 @@ class IAST;
* that contain only compatible expressions.
*
* Compatible expressions are comparisons of identifiers, constants, and logical operations on them.
*
* Throws INCORRECT_QUERY if external_table_strict_query (from context settings)
* is set and some expression from WHERE is not compatible.
*/
String transformQueryForExternalDatabase(
const SelectQueryInfo & query_info,

View File

@ -12,6 +12,15 @@ class Context;
*/
class ITableFunctionFileLike : public ITableFunction
{
protected:
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
String filename;
String format;
String structure;
String compression_method = "auto";
private:
StoragePtr executeImpl(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override;
@ -21,13 +30,6 @@ private:
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
bool hasStaticStructure() const override { return true; }
String filename;
String format;
String structure;
String compression_method = "auto";
};
}

View File

@ -34,8 +34,6 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int UNKNOWN_TABLE;
}
@ -46,46 +44,19 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr
if (!args_func.arguments)
throw Exception("Table function 'mysql' must have arguments.", ErrorCodes::LOGICAL_ERROR);
ASTs & args = args_func.arguments->children;
if (args.size() < 5 || args.size() > 7)
throw Exception("Table function 'mysql' requires 5-7 parameters: MySQL('host:port', database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
String host_port = args[0]->as<ASTLiteral &>().value.safeGet<String>();
remote_database_name = args[1]->as<ASTLiteral &>().value.safeGet<String>();
remote_table_name = args[2]->as<ASTLiteral &>().value.safeGet<String>();
user_name = args[3]->as<ASTLiteral &>().value.safeGet<String>();
password = args[4]->as<ASTLiteral &>().value.safeGet<String>();
/// Split into replicas if needed. 3306 is the default MySQL port number
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
pool.emplace(remote_database_name, addresses, user_name, password);
if (args.size() >= 6)
replace_query = args[5]->as<ASTLiteral &>().value.safeGet<UInt64>() > 0;
if (args.size() == 7)
on_duplicate_clause = args[6]->as<ASTLiteral &>().value.safeGet<String>();
if (replace_query && !on_duplicate_clause.empty())
throw Exception(
"Only one of 'replace_query' and 'on_duplicate_clause' can be specified, or none of them",
ErrorCodes::BAD_ARGUMENTS);
configuration = StorageMySQL::getConfiguration(args_func.arguments->children, context);
pool.emplace(configuration->database, configuration->addresses, configuration->username, configuration->password);
}
ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr context) const
{
const auto & settings = context->getSettingsRef();
const auto tables_and_columns = fetchTablesColumnsList(*pool, remote_database_name, {remote_table_name}, settings, settings.mysql_datatypes_support_level);
const auto tables_and_columns = fetchTablesColumnsList(*pool, configuration->database, {configuration->table}, settings, settings.mysql_datatypes_support_level);
const auto columns = tables_and_columns.find(remote_table_name);
const auto columns = tables_and_columns.find(configuration->table);
if (columns == tables_and_columns.end())
throw Exception("MySQL table " + (remote_database_name.empty() ? "" : (backQuote(remote_database_name) + "."))
+ backQuote(remote_table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
throw Exception("MySQL table " + (configuration->database.empty() ? "" : (backQuote(configuration->database) + "."))
+ backQuote(configuration->table) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
return columns->second;
}
@ -101,10 +72,10 @@ StoragePtr TableFunctionMySQL::executeImpl(
auto res = StorageMySQL::create(
StorageID(getDatabaseName(), table_name),
std::move(*pool),
remote_database_name,
remote_table_name,
replace_query,
on_duplicate_clause,
configuration->database,
configuration->table,
configuration->replace_query,
configuration->on_duplicate_clause,
columns,
ConstraintsDescription{},
String{},

View File

@ -5,6 +5,7 @@
#if USE_MYSQL
#include <TableFunctions/ITableFunction.h>
#include <Storages/ExternalDataSourceConfiguration.h>
#include <mysqlxx/Pool.h>
@ -30,14 +31,8 @@ private:
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
String remote_database_name;
String remote_table_name;
String user_name;
String password;
bool replace_query = false;
String on_duplicate_clause;
mutable std::optional<mysqlxx::PoolWithFailover> pool;
std::optional<StorageMySQLConfiguration> configuration;
};
}

View File

@ -21,7 +21,6 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
@ -33,12 +32,12 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/,
auto result = std::make_shared<StoragePostgreSQL>(
StorageID(getDatabaseName(), table_name),
connection_pool,
remote_table_name,
configuration->table,
columns,
ConstraintsDescription{},
String{},
remote_table_schema,
on_conflict);
configuration->schema,
configuration->on_conflict);
result->startup();
return result;
@ -51,8 +50,8 @@ ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr c
auto connection_holder = connection_pool->get();
auto columns = fetchPostgreSQLTableStructure(
connection_holder->get(),
remote_table_schema.empty() ? doubleQuoteString(remote_table_name)
: doubleQuoteString(remote_table_schema) + '.' + doubleQuoteString(remote_table_name),
configuration->schema.empty() ? doubleQuoteString(configuration->table)
: doubleQuoteString(configuration->schema) + '.' + doubleQuoteString(configuration->table),
use_nulls).columns;
return ColumnsDescription{*columns};
@ -62,37 +61,13 @@ ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr c
void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
const auto & func_args = ast_function->as<ASTFunction &>();
if (!func_args.arguments)
throw Exception("Table function 'PostgreSQL' must have arguments.", ErrorCodes::BAD_ARGUMENTS);
ASTs & args = func_args.arguments->children;
if (args.size() < 5 || args.size() > 7)
throw Exception("Table function 'PostgreSQL' requires from 5 to 7 parameters: "
"PostgreSQL('host:port', 'database', 'table', 'user', 'password', [, 'schema', 'ON CONFLICT ...']).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
/// Split into replicas if needed. 5432 is a default postgresql port.
const auto & host_port = args[0]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
remote_table_name = args[2]->as<ASTLiteral &>().value.safeGet<String>();
if (args.size() >= 6)
remote_table_schema = args[5]->as<ASTLiteral &>().value.safeGet<String>();
if (args.size() >= 7)
on_conflict = args[6]->as<ASTLiteral &>().value.safeGet<String>();
connection_pool = std::make_shared<postgres::PoolWithFailover>(
args[1]->as<ASTLiteral &>().value.safeGet<String>(),
addresses,
args[3]->as<ASTLiteral &>().value.safeGet<String>(),
args[4]->as<ASTLiteral &>().value.safeGet<String>());
configuration.emplace(StoragePostgreSQL::getConfiguration(func_args.arguments->children, context));
connection_pool = std::make_shared<postgres::PoolWithFailover>(*configuration,
context->getSettingsRef().postgresql_connection_pool_size,
context->getSettingsRef().postgresql_connection_pool_wait_timeout);
}

View File

@ -6,6 +6,7 @@
#if USE_LIBPQXX
#include <TableFunctions/ITableFunction.h>
#include <Core/PostgreSQL/PoolWithFailover.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace DB
@ -27,9 +28,8 @@ private:
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
String connection_str;
String remote_table_name, remote_table_schema, on_conflict;
postgres::PoolWithFailoverPtr connection_pool;
std::optional<StoragePostgreSQLConfiguration> configuration;
};
}

View File

@ -3,13 +3,13 @@
#if USE_AWS_S3
#include <IO/S3Common.h>
#include <Storages/StorageS3.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionS3.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Storages/StorageS3.h>
#include "registerTableFunctions.h"
@ -38,51 +38,75 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTs & args = args_func.at(0)->children;
StorageS3Configuration configuration;
if (args.size() < 3 || args.size() > 6)
throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
/// Size -> argument indexes
static auto size_to_args = std::map<size_t, std::map<String, size_t>>
if (auto named_collection = getURLBasedDataSourceConfiguration(args, context))
{
{3, {{"format", 1}, {"structure", 2}}},
{4, {{"format", 1}, {"structure", 2}, {"compression_method", 3}}},
{5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}},
{6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}}
};
auto [common_configuration, storage_specific_args] = named_collection.value();
configuration.set(common_configuration);
/// This argument is always the first
filename = args[0]->as<ASTLiteral &>().value.safeGet<String>();
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
if (arg_name == "access_key_id")
configuration.access_key_id = arg_value.safeGet<String>();
else if (arg_name == "secret_access_key")
configuration.secret_access_key = arg_value.safeGet<String>();
else
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Unknown key-value argument `{}` for StorageS3, expected: "
"url, [access_key_id, secret_access_key], name of used format, structure and [compression_method].",
arg_name);
}
}
else
{
if (args.size() < 3 || args.size() > 6)
throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
auto & args_to_idx = size_to_args[args.size()];
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
if (args_to_idx.contains("format"))
format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
/// Size -> argument indexes
static auto size_to_args = std::map<size_t, std::map<String, size_t>>
{
{3, {{"format", 1}, {"structure", 2}}},
{4, {{"format", 1}, {"structure", 2}, {"compression_method", 3}}},
{5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}},
{6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}}
};
if (args_to_idx.contains("structure"))
structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
/// This argument is always the first
configuration.url = args[0]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("compression_method"))
compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
auto & args_to_idx = size_to_args[args.size()];
if (args_to_idx.contains("access_key_id"))
access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("format"))
configuration.format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("secret_access_key"))
secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("structure"))
configuration.structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("compression_method"))
configuration.compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("access_key_id"))
configuration.access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("secret_access_key"))
configuration.secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
}
s3_configuration = std::move(configuration);
}
ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const
{
return parseColumnsListFromString(structure, context);
return parseColumnsListFromString(s3_configuration->structure, context);
}
StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
Poco::URI uri (filename);
Poco::URI uri (s3_configuration->url);
S3::URI s3_uri (uri);
UInt64 max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries;
UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size;
@ -91,10 +115,10 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context
StoragePtr storage = StorageS3::create(
s3_uri,
access_key_id,
secret_access_key,
s3_configuration->access_key_id,
s3_configuration->secret_access_key,
StorageID(getDatabaseName(), table_name),
format,
s3_configuration->format,
max_single_read_retries,
min_upload_part_size,
max_single_part_upload_size,
@ -105,7 +129,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, Context
context,
/// No format_settings for table function S3
std::nullopt,
compression_method);
s3_configuration->compression_method);
storage->startup();

View File

@ -5,6 +5,7 @@
#if USE_AWS_S3
#include <TableFunctions/ITableFunction.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace DB
@ -36,12 +37,7 @@ protected:
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
String filename;
String format;
String structure;
String access_key_id;
String secret_access_key;
String compression_method = "auto";
std::optional<StorageS3Configuration> s3_configuration;
};
class TableFunctionCOS : public TableFunctionS3

View File

@ -4,6 +4,7 @@
#include <Access/AccessFlags.h>
#include <Formats/FormatFactory.h>
#include <Poco/URI.h>
#include <Parsers/ASTFunction.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/StorageURL.h>
#include <TableFunctions/TableFunctionFactory.h>
@ -12,6 +13,48 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
const auto & func_args = ast_function->as<ASTFunction &>();
if (!func_args.arguments)
throw Exception("Table function 'URL' must have arguments.", ErrorCodes::BAD_ARGUMENTS);
URLBasedDataSourceConfiguration configuration;
if (auto with_named_collection = getURLBasedDataSourceConfiguration(func_args.arguments->children, context))
{
auto [common_configuration, storage_specific_args] = with_named_collection.value();
configuration.set(common_configuration);
if (!storage_specific_args.empty())
{
String illegal_args;
for (const auto & arg : storage_specific_args)
{
if (!illegal_args.empty())
illegal_args += ", ";
illegal_args += arg.first;
}
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguments {} for table function URL", illegal_args);
}
filename = configuration.url;
format = configuration.format;
structure = configuration.structure;
compression_method = configuration.compression_method;
}
else
{
ITableFunctionFileLike::parseArguments(ast_function, context);
}
}
StoragePtr TableFunctionURL::getStorage(
const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context,
const std::string & table_name, const String & compression_method_) const

View File

@ -19,6 +19,9 @@ public:
return name;
}
protected:
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
private:
StoragePtr getStorage(
const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context,

View File

@ -704,6 +704,18 @@
"clang-tidy": "disable",
"with_coverage": false
}
},
"ClickHouse Keeper Jepsen": {
"required_build_properties": {
"compiler": "clang-13",
"package_type": "binary",
"build_type": "relwithdebuginfo",
"sanitizer": "none",
"bundled": "bundled",
"splitted": "unsplitted",
"clang-tidy": "disable",
"with_coverage": false
}
}
}
}

View File

@ -817,13 +817,13 @@ class TestSuite:
stop_time = None
exit_code = multiprocessing.Value("i", 0)
server_died = multiprocessing.Event()
stop_tests_triggered_lock = multiprocessing.Lock()
stop_tests_triggered = multiprocessing.Event()
queue = multiprocessing.Queue(maxsize=1)
multiprocessing_manager = multiprocessing.Manager()
restarted_tests = multiprocessing_manager.list()
exit_code = None
server_died = None
stop_tests_triggered_lock = None
stop_tests_triggered = None
queue = None
multiprocessing_manager = None
restarted_tests = None
# def run_tests_array(all_tests: List[str], num_tests: int, test_suite: TestSuite):
def run_tests_array(all_tests_with_params):
@ -1308,6 +1308,15 @@ def get_additional_client_options_url(args):
if __name__ == '__main__':
stop_time = None
exit_code = multiprocessing.Value("i", 0)
server_died = multiprocessing.Event()
stop_tests_triggered_lock = multiprocessing.Lock()
stop_tests_triggered = multiprocessing.Event()
queue = multiprocessing.Queue(maxsize=1)
multiprocessing_manager = multiprocessing.Manager()
restarted_tests = multiprocessing_manager.list()
# Move to a new process group and kill it at exit so that we don't have any
# infinite tests processes left
# (new process group is required to avoid killing some parent processes)

View File

@ -760,7 +760,7 @@ class ClickHouseCluster:
hostname=None, env_variables=None, image="clickhouse/integration-test", tag=None,
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None,
zookeeper_docker_compose_path=None, minio_certs_dir=None, use_keeper=True,
main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True):
main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="yandex"):
"""Add an instance to the cluster.
@ -832,7 +832,8 @@ class ClickHouseCluster:
main_config_name=main_config_name,
users_config_name=users_config_name,
copy_common_configs=copy_common_configs,
tmpfs=tmpfs or [])
tmpfs=tmpfs or [],
config_root_name=config_root_name)
docker_compose_yml_dir = get_docker_compose_path()
@ -1802,7 +1803,7 @@ class ClickHouseInstance:
main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True,
hostname=None, env_variables=None,
image="clickhouse/integration-test", tag="latest",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None):
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None, config_root_name="yandex"):
self.name = name
self.base_cmd = cluster.base_cmd
@ -1875,6 +1876,7 @@ class ClickHouseInstance:
self.ipv6_address = ipv6_address
self.with_installed_binary = with_installed_binary
self.is_up = False
self.config_root_name = config_root_name
def is_built_with_sanitizer(self, sanitizer_name=''):
@ -2219,9 +2221,8 @@ class ClickHouseInstance:
finally:
sock.close()
@staticmethod
def dict_to_xml(dictionary):
xml_str = dict2xml(dictionary, wrap="clickhouse", indent=" ", newlines=True)
def dict_to_xml(self, dictionary):
xml_str = dict2xml(dictionary, wrap=self.config_root_name, indent=" ", newlines=True)
return xml_str
@property
@ -2304,15 +2305,22 @@ class ClickHouseInstance:
dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries'))
os.mkdir(dictionaries_dir)
def write_embedded_config(name, dest_dir):
with open(p.join(HELPERS_DIR, name), 'r') as f:
data = f.read()
data = data.replace('yandex', self.config_root_name)
with open(p.join(dest_dir, name), 'w') as r:
r.write(data)
logging.debug("Copy common configuration from helpers")
# The file is named with 0_ prefix to be processed before other configuration overloads.
if self.copy_common_configs:
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir)
write_embedded_config('0_common_instance_config.xml', self.config_d_dir)
write_embedded_config('0_common_instance_users.xml', users_d_dir)
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir)
if len(self.custom_dictionaries_paths):
shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir)
write_embedded_config('0_common_enable_dictionaries.xml', self.config_d_dir)
logging.debug("Generate and write macros file")
macros = self.macros.copy()

View File

@ -2,6 +2,6 @@ user_directories:
users_xml:
path: users.xml
local_directory:
path: access/
path: /var/lib/clickhouse/access/
"@replace": replace

View File

@ -1,18 +1,18 @@
path:
- ./
- /var/lib/clickhouse
- "@replace": replace
tmp_path:
- ./tmp/
- /var/lib/clickhouse/tmp/
- "@replace": replace
user_files_path:
- ./user_files/
- /var/lib/clickhouse/user_files/
- "@replace": replace
format_schema_path:
- ./format_schemas/
- /var/lib/clickhouse/format_schemas/
- "@replace": replace
access_control_path:
- ./access/
- /var/lib/clickhouse/access/
- "@replace": replace
top_level_domains_path:
- ./top_level_domains/
- /var/lib/clickhouse/top_level_domains/
- "@replace": replace

View File

@ -64,10 +64,10 @@
<mmap_cache_size>1000</mmap_cache_size>
<path>/var/lib/clickhouse/</path>
<path>./</path>
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
<tmp_path>./tmp/</tmp_path>
<user_files_path>./user_files/</user_files_path>
<ldap_servers>
</ldap_servers>
<user_directories>

View File

@ -32,7 +32,7 @@ def test_xml_main_conf():
all_userd = ['configs/users.d/allow_introspection_functions.yaml',
'configs/users.d/log_queries.yaml']
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd, with_zookeeper=False)
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd, with_zookeeper=False, config_root_name='clickhouse')
try:
cluster.start()

View File

@ -2,6 +2,6 @@ user_directories:
users_xml:
path: users.yaml
local_directory:
path: access/
path: /var/lib/clickhouse/access/
"@replace": replace

View File

@ -1,18 +1,18 @@
path:
- ./
- /var/lib/clickhouse
- "@replace": replace
tmp_path:
- ./tmp/
- /var/lib/clickhouse/tmp/
- "@replace": replace
user_files_path:
- ./user_files/
- /var/lib/clickhouse/user_files/
- "@replace": replace
format_schema_path:
- ./format_schemas/
- /var/lib/clickhouse/format_schemas/
- "@replace": replace
access_control_path:
- ./access/
- /var/lib/clickhouse/access/
- "@replace": replace
top_level_domains_path:
- ./top_level_domains/
- /var/lib/clickhouse/top_level_domains/
- "@replace": replace

View File

@ -64,10 +64,10 @@
<mmap_cache_size>1000</mmap_cache_size>
<path>/var/lib/clickhouse/</path>
<path>./</path>
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
<tmp_path>./tmp/</tmp_path>
<user_files_path>./user_files/</user_files_path>
<ldap_servers>
</ldap_servers>
<user_directories>

View File

@ -32,7 +32,7 @@ def test_extra_yaml_mix():
'configs/users.d/log_queries.yaml']
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd, with_zookeeper=False,
users_config_name="users.yaml", copy_common_configs=False)
users_config_name="users.yaml", copy_common_configs=False, config_root_name="clickhouse")
try:
cluster.start()

View File

@ -2,6 +2,6 @@ user_directories:
users_xml:
path: users.yaml
local_directory:
path: access/
path: /var/lib/clickhouse/access/
"@replace": replace

View File

@ -1,18 +1,18 @@
path:
- ./
- /var/lib/clickhouse
- "@replace": replace
tmp_path:
- ./tmp/
- /var/lib/clickhouse/tmp/
- "@replace": replace
user_files_path:
- ./user_files/
- /var/lib/clickhouse/user_files/
- "@replace": replace
format_schema_path:
- ./format_schemas/
- /var/lib/clickhouse/format_schemas/
- "@replace": replace
access_control_path:
- ./access/
- /var/lib/clickhouse/access/
- "@replace": replace
top_level_domains_path:
- ./top_level_domains/
- /var/lib/clickhouse/top_level_domains/
- "@replace": replace

View File

@ -48,9 +48,9 @@ total_memory_tracker_sample_probability: 0
uncompressed_cache_size: 8589934592
mark_cache_size: 5368709120
mmap_cache_size: 1000
path: /var/lib/clickhouse/
tmp_path: /var/lib/clickhouse/tmp/
user_files_path: /var/lib/clickhouse/user_files/
path: ./
tmp_path: ./tmp
user_files_path: ./user_files/
ldap_servers: ''
user_directories:
users_xml:

View File

@ -31,7 +31,7 @@ def test_yaml_full_conf():
'configs/users.d/log_queries.yaml']
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd,
with_zookeeper=False, main_config_name="config.yaml", users_config_name="users.yaml", copy_common_configs=False)
with_zookeeper=False, main_config_name="config.yaml", users_config_name="users.yaml", copy_common_configs=False, config_root_name="clickhouse")
try:
cluster.start()

View File

@ -7,7 +7,7 @@
</users_xml>
<local_directory>
<!-- Path to folder where users created by SQL commands are stored. -->
<path>access/</path>
<path>/var/lib/clickhouse/access/</path>
</local_directory>
</user_directories>
</clickhouse>

View File

@ -1,8 +1,8 @@
<clickhouse>
<path replace="replace">./</path>
<tmp_path replace="replace">./tmp/</tmp_path>
<user_files_path replace="replace">./user_files/</user_files_path>
<format_schema_path replace="replace">./format_schemas/</format_schema_path>
<access_control_path replace="replace">./access/</access_control_path>
<top_level_domains_path replace="replace">./top_level_domains/</top_level_domains_path>
<path replace="replace">/var/lib/clickhouse</path>
<tmp_path replace="replace">/var/lib/clickhouse/tmp/</tmp_path>
<user_files_path replace="replace">/var/lib/clickhouse/user_files/</user_files_path>
<format_schema_path replace="replace">/var/lib/clickhouse/format_schemas/</format_schema_path>
<access_control_path replace="replace">/var/lib/clickhouse/access/</access_control_path>
<top_level_domains_path replace="replace">/var/lib/clickhouse/top_level_domains/</top_level_domains_path>
</clickhouse>

View File

@ -48,9 +48,9 @@ total_memory_tracker_sample_probability: 0
uncompressed_cache_size: 8589934592
mark_cache_size: 5368709120
mmap_cache_size: 1000
path: /var/lib/clickhouse/
tmp_path: /var/lib/clickhouse/tmp/
user_files_path: /var/lib/clickhouse/user_files/
path: ./
tmp_path: ./tmp/
user_files_path: ./user_files/
ldap_servers: ''
user_directories:
users_xml:

View File

@ -32,7 +32,8 @@ def test_yaml_main_conf():
'configs/users.d/log_queries.xml']
node = cluster.add_instance('node', base_config_dir='configs', main_configs=all_confd, user_configs=all_userd,
with_zookeeper=False, main_config_name="config.yaml", users_config_name="users.yaml", copy_common_configs=False)
with_zookeeper=False, main_config_name="config.yaml", users_config_name="users.yaml",
copy_common_configs=False, config_root_name="clickhouse")
try:
cluster.start()

View File

@ -0,0 +1,25 @@
<yandex>
<named_collections>
<mysql1>
<user>root</user>
<password>clickhouse</password>
<host>mysql57</host>
<port>3306</port>
<database>test</database>
<table>test_table</table>
</mysql1>
<mysql2>
<user>postgres</user>
<password>mysecretpassword</password>
<host>postgres1</host>
</mysql2>
<mysql3>
<user>root</user>
<password>clickhouse</password>
<host>mysql57</host>
<port>1111</port>
<database>test</database>
<table>test_table</table>
</mysql3>
</named_collections>
</yandex>

View File

@ -7,7 +7,7 @@ import time
import logging
DICTS = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml']
CONFIG_FILES = ['configs/remote_servers.xml']
CONFIG_FILES = ['configs/remote_servers.xml', 'configs/named_collections.xml']
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', main_configs=CONFIG_FILES, with_mysql=True, dictionaries=DICTS)
@ -157,6 +157,55 @@ def test_mysql_dictionaries_custom_query_partial_load_complex_key(started_cluste
execute_mysql_query(mysql_connection, "DROP TABLE test.test_table_2;")
def test_predefined_connection_configuration(started_cluster):
mysql_connection = get_mysql_conn(started_cluster)
execute_mysql_query(mysql_connection, "DROP TABLE IF EXISTS test.test_table")
execute_mysql_query(mysql_connection, "CREATE TABLE IF NOT EXISTS test.test_table (id Integer, value Integer);")
execute_mysql_query(mysql_connection, "INSERT INTO test.test_table VALUES (100, 200);")
instance.query('''
DROP DICTIONARY IF EXISTS dict;
CREATE DICTIONARY dict (id UInt32, value UInt32)
PRIMARY KEY id
SOURCE(MYSQL(NAME mysql1))
LIFETIME(MIN 1 MAX 2)
LAYOUT(HASHED());
''')
result = instance.query("SELECT dictGetUInt32(dict, 'value', toUInt64(100))")
assert(int(result) == 200)
instance.query('''
DROP DICTIONARY dict;
CREATE DICTIONARY dict (id UInt32, value UInt32)
PRIMARY KEY id
SOURCE(MYSQL(NAME mysql2))
LIFETIME(MIN 1 MAX 2)
LAYOUT(HASHED());
''')
result = instance.query_and_get_error("SELECT dictGetUInt32(dict, 'value', toUInt64(100))")
instance.query('''
DROP DICTIONARY dict;
CREATE DICTIONARY dict (id UInt32, value UInt32)
PRIMARY KEY id
SOURCE(MYSQL(NAME unknown_collection))
LIFETIME(MIN 1 MAX 2)
LAYOUT(HASHED());
''')
result = instance.query_and_get_error("SELECT dictGetUInt32(dict, 'value', toUInt64(100))")
instance.query('''
DROP DICTIONARY dict;
CREATE DICTIONARY dict (id UInt32, value UInt32)
PRIMARY KEY id
SOURCE(MYSQL(NAME mysql3 PORT 3306))
LIFETIME(MIN 1 MAX 2)
LAYOUT(HASHED());
''')
result = instance.query("SELECT dictGetUInt32(dict, 'value', toUInt64(100))")
assert(int(result) == 200)
def create_mysql_db(mysql_connection, name):
with mysql_connection.cursor() as cursor:
cursor.execute("DROP DATABASE IF EXISTS {}".format(name))

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