Merge branch 'master' into trying_actions

This commit is contained in:
mergify[bot] 2021-09-29 11:37:44 +00:00 committed by GitHub
commit 3caa9c0430
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
112 changed files with 3375 additions and 1112 deletions

View File

@ -1,4 +1,4 @@
cmake_minimum_required(VERSION 3.3) cmake_minimum_required(VERSION 3.14)
foreach(policy foreach(policy
CMP0023 CMP0023

View File

@ -6,38 +6,6 @@ Thank you.
## Technical Info ## Technical Info
We have a [developer's guide](https://clickhouse.yandex/docs/en/development/developer_instruction/) for writing code for ClickHouse. Besides this guide, you can find [Overview of ClickHouse Architecture](https://clickhouse.yandex/docs/en/development/architecture/) and instructions on how to build ClickHouse in different environments. We have a [developer's guide](https://clickhouse.com/docs/en/development/developer_instruction/) for writing code for ClickHouse. Besides this guide, you can find [Overview of ClickHouse Architecture](https://clickhouse.com/docs/en/development/architecture/) and instructions on how to build ClickHouse in different environments.
If you want to contribute to documentation, read the [Contributing to ClickHouse Documentation](docs/README.md) guide. If you want to contribute to documentation, read the [Contributing to ClickHouse Documentation](docs/README.md) guide.
## Legal Info
In order for us (YANDEX LLC) to accept patches and other contributions from you, you may adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here:
1) https://yandex.ru/legal/cla/?lang=en (in English) and
2) https://yandex.ru/legal/cla/?lang=ru (in Russian).
By adopting the CLA, you state the following:
* You obviously wish and are willingly licensing your contributions to us for our open source projects under the terms of the CLA,
* You have read the terms and conditions of the CLA and agree with them in full,
* You are legally able to provide and license your contributions as stated,
* We may use your contributions for our open source projects and for any other our project too,
* We rely on your assurances concerning the rights of third parties in relation to your contributions.
If you agree with these principles, please read and adopt our CLA. By providing us your contributions, you hereby declare that you have already read and adopt our CLA, and we may freely merge your contributions with our corresponding open source project and use it in further in accordance with terms and conditions of the CLA.
If you have already adopted terms and conditions of the CLA, you are able to provide your contributes. When you submit your pull request, please add the following information into it:
```
I hereby agree to the terms of the CLA available at: [link].
```
Replace the bracketed text as follows:
* [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian).
It is enough to provide us such notification once.
As an alternative, you can provide DCO instead of CLA. You can find the text of DCO here: https://developercertificate.org/
It is enough to read and copy it verbatim to your pull request.
If you don't agree with the CLA and don't want to provide DCO, you still can open a pull request to provide your contributions.

1
debian/control vendored
View File

@ -7,6 +7,7 @@ Build-Depends: debhelper (>= 9),
ninja-build, ninja-build,
clang-13, clang-13,
llvm-13, llvm-13,
lld-13,
libc6-dev, libc6-dev,
tzdata tzdata
Standards-Version: 3.9.8 Standards-Version: 3.9.8

View File

@ -59,9 +59,68 @@ A lambda function that accepts multiple arguments can also be passed to a higher
For some functions the first argument (the lambda function) can be omitted. In this case, identical mapping is assumed. For some functions the first argument (the lambda function) can be omitted. In this case, identical mapping is assumed.
## User Defined Functions {#user-defined-functions} ## SQL User Defined Functions {#user-defined-functions}
Custom functions from lambda expressions can be created using the [CREATE FUNCTION](../statements/create/function.md) statement. To delete these functions use the [DROP FUNCTION](../statements/drop.md#drop-function) statement.
## Executable User Defined Functions {#executable-user-defined-functions}
ClickHouse can call any external executable program or script to process data. Describe such functions in a [configuration file](../../operations/configuration-files.md) and add the path of that file to the main configuration in `user_defined_executable_functions_config` setting. If a wildcard symbol `*` is used in the path, then all files matching the pattern are loaded. Example:
``` xml
<user_defined_executable_functions_config>*_function.xml</user_defined_executable_functions_config>
```
User defined function configurations are searched relative to the path specified in the `user_files_path` setting.
A function configuration contains the following settings:
- `name` - a function name.
- `command` - a command or a script to execute.
- `argument` - argument description with the `type` of an argument. Each argument is described in a separate setting.
- `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command.
- `return_type` - the type of a returned value.
- `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created.
- `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`.
- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`.
- `pool_size` - the size of a command pool. Optional. Default value is `16`.
- `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded.
- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`.
The command must read arguments from `STDIN` and must output the result to `STDOUT`. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk.
**Example**
Creating `test_function` using XML configuration:
```
<functions>
<function>
<type>executable</type>
<name>test_function</name>
<return_type>UInt64</return_type>
<argument>
<type>UInt64</type>
</argument>
<argument>
<type>UInt64</type>
</argument>
<format>TabSeparated</format>
<command>cd /; clickhouse-local --input-format TabSeparated --output-format TabSeparated --structure 'x UInt64, y UInt64' --query "SELECT x + y FROM table"</command>
<lifetime>0</lifetime>
</function>
</functions>
```
Query:
``` sql
SELECT test_function(toUInt64(2), toUInt64(2));
```
Result:
``` text
┌─test_function(toUInt64(2), toUInt64(2))─┐
│ 4 │
└─────────────────────────────────────────┘
```
Custom functions can be created using the [CREATE FUNCTION](../statements/create/function.md) statement. To delete these functions use the [DROP FUNCTION](../statements/drop.md#drop-function) statement.
## Error Handling {#error-handling} ## Error Handling {#error-handling}

View File

@ -155,6 +155,8 @@ Hierarchy of privileges:
- `SYSTEM RELOAD CONFIG` - `SYSTEM RELOAD CONFIG`
- `SYSTEM RELOAD DICTIONARY` - `SYSTEM RELOAD DICTIONARY`
- `SYSTEM RELOAD EMBEDDED DICTIONARIES` - `SYSTEM RELOAD EMBEDDED DICTIONARIES`
- `SYSTEM RELOAD FUNCTION`
- `SYSTEM RELOAD FUNCTIONS`
- `SYSTEM MERGES` - `SYSTEM MERGES`
- `SYSTEM TTL MERGES` - `SYSTEM TTL MERGES`
- `SYSTEM FETCHES` - `SYSTEM FETCHES`

View File

@ -12,6 +12,8 @@ The list of available `SYSTEM` statements:
- [RELOAD DICTIONARY](#query_language-system-reload-dictionary) - [RELOAD DICTIONARY](#query_language-system-reload-dictionary)
- [RELOAD MODELS](#query_language-system-reload-models) - [RELOAD MODELS](#query_language-system-reload-models)
- [RELOAD MODEL](#query_language-system-reload-model) - [RELOAD MODEL](#query_language-system-reload-model)
- [RELOAD FUNCTIONS](#query_language-system-reload-functions)
- [RELOAD FUNCTION](#query_language-system-reload-functions)
- [DROP DNS CACHE](#query_language-system-drop-dns-cache) - [DROP DNS CACHE](#query_language-system-drop-dns-cache)
- [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [DROP MARK CACHE](#query_language-system-drop-mark-cache)
- [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache) - [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache)
@ -83,6 +85,17 @@ Completely reloads a CatBoost model `model_name` if the configuration was update
SYSTEM RELOAD MODEL <model_name> SYSTEM RELOAD MODEL <model_name>
``` ```
## RELOAD FUNCTIONS {#query_language-system-reload-functions}
Reloads all registered [executable user defined functions](../functions/index.md#executable-user-defined-functions) or one of them from a configuration file.
**Syntax**
```sql
RELOAD FUNCTIONS
RELOAD FUNCTION function_name
```
## DROP DNS CACHE {#query_language-system-drop-dns-cache} ## DROP DNS CACHE {#query_language-system-drop-dns-cache}
Resets ClickHouses internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries). Resets ClickHouses internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries).

View File

@ -58,9 +58,68 @@ str -> str != Referer
Для некоторых функций первый аргумент (лямбда-функция) может отсутствовать. В этом случае подразумевается тождественное отображение. Для некоторых функций первый аргумент (лямбда-функция) может отсутствовать. В этом случае подразумевается тождественное отображение.
## Пользовательские функции {#user-defined-functions} ## Пользовательские функции SQL {#user-defined-functions}
Функции можно создавать с помощью выражения [CREATE FUNCTION](../statements/create/function.md). Для удаления таких функций используется выражение [DROP FUNCTION](../statements/drop.md#drop-function). Функции можно создавать из лямбда выражений с помощью [CREATE FUNCTION](../statements/create/function.md). Для удаления таких функций используется выражение [DROP FUNCTION](../statements/drop.md#drop-function).
## Исполняемые пользовательские функции {#executable-user-defined-functions}
ClickHouse может вызывать внешнюю программу или скрипт для обработки данных. Такие функции описываются в [конфигурационном файле](../../operations/configuration-files.md). Путь к нему должен быть указан в настройке `user_defined_executable_functions_config` в основной конфигурации. В пути можно использовать символ подстановки `*`, тогда будут загружены все файлы, соответствующие шаблону. Пример:
``` xml
<user_defined_executable_functions_config>*_function.xml</user_defined_executable_functions_config>
```
Файлы с описанием функций ищутся относительно каталога, заданного в настройке `user_files_path`.
Конфигурация функции содержит следующие настройки:
- `name` - имя функции.
- `command` - исполняемая команда или скрипт.
- `argument` - описание аргумента, содержащее его тип во вложенной настройке `type`. Каждый аргумент описывается отдельно.
- `format` - [формат](../../interfaces/formats.md) передачи аргументов.
- `return_type` - тип возвращаемого значения.
- `type` - вариант запуска команды. Если задан вариант `executable`, то запускается одна команда. При указании `executable_pool` создается пул команд.
- `max_command_execution_time` - максимальное время в секундах, которое отводится на обработку блока данных. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`.
- `command_termination_timeout` - максимальное время завершения команды в секундах после закрытия конвейера. Если команда не завершается, то процессу отправляется сигнал `SIGTERM`. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`.
- `pool_size` - размер пула команд. Необязательная настройка. Значение по умолчанию `16`.
- `lifetime` - интервал перезагрузки функций в секундах. Если задан `0`, то функция не перезагружается.
- `send_chunk_header` - управляет отправкой количества строк перед отправкой блока данных для обработки. Необязательная настройка. Значение по умолчанию `false`.
Команда должна читать аргументы из `STDIN` и выводить результат в `STDOUT`. Обработка должна выполняться в цикле. То есть после обработки группы аргументов команда должна ожидать следующую группу.
**Пример**
XML конфигурация, описывающая функцию `test_function`:
```
<functions>
<function>
<type>executable</type>
<name>test_function</name>
<return_type>UInt64</return_type>
<argument>
<type>UInt64</type>
</argument>
<argument>
<type>UInt64</type>
</argument>
<format>TabSeparated</format>
<command>cd /; clickhouse-local --input-format TabSeparated --output-format TabSeparated --structure 'x UInt64, y UInt64' --query "SELECT x + y FROM table"</command>
<lifetime>0</lifetime>
</function>
</functions>
```
Запрос:
``` sql
SELECT test_function(toUInt64(2), toUInt64(2));
```
Результат:
``` text
┌─test_function(toUInt64(2), toUInt64(2))─┐
│ 4 │
└─────────────────────────────────────────┘
```
## Обработка ошибок {#obrabotka-oshibok} ## Обработка ошибок {#obrabotka-oshibok}

View File

@ -157,6 +157,8 @@ GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION
- `SYSTEM RELOAD CONFIG` - `SYSTEM RELOAD CONFIG`
- `SYSTEM RELOAD DICTIONARY` - `SYSTEM RELOAD DICTIONARY`
- `SYSTEM RELOAD EMBEDDED DICTIONARIES` - `SYSTEM RELOAD EMBEDDED DICTIONARIES`
- `SYSTEM RELOAD FUNCTION`
- `SYSTEM RELOAD FUNCTIONS`
- `SYSTEM MERGES` - `SYSTEM MERGES`
- `SYSTEM TTL MERGES` - `SYSTEM TTL MERGES`
- `SYSTEM FETCHES` - `SYSTEM FETCHES`

View File

@ -10,6 +10,8 @@ toc_title: SYSTEM
- [RELOAD DICTIONARY](#query_language-system-reload-dictionary) - [RELOAD DICTIONARY](#query_language-system-reload-dictionary)
- [RELOAD MODELS](#query_language-system-reload-models) - [RELOAD MODELS](#query_language-system-reload-models)
- [RELOAD MODEL](#query_language-system-reload-model) - [RELOAD MODEL](#query_language-system-reload-model)
- [RELOAD FUNCTIONS](#query_language-system-reload-functions)
- [RELOAD FUNCTION](#query_language-system-reload-functions)
- [DROP DNS CACHE](#query_language-system-drop-dns-cache) - [DROP DNS CACHE](#query_language-system-drop-dns-cache)
- [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [DROP MARK CACHE](#query_language-system-drop-mark-cache)
- [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache) - [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache)
@ -80,6 +82,17 @@ SYSTEM RELOAD MODELS
SYSTEM RELOAD MODEL <model_name> SYSTEM RELOAD MODEL <model_name>
``` ```
## RELOAD FUNCTIONS {#query_language-system-reload-functions}
Перезагружает все зарегистрированные [исполняемые пользовательские функции](../functions/index.md#executable-user-defined-functions) или одну из них из файла конфигурации.
**Синтаксис**
```sql
RELOAD FUNCTIONS
RELOAD FUNCTION function_name
```
## DROP DNS CACHE {#query_language-system-drop-dns-cache} ## DROP DNS CACHE {#query_language-system-drop-dns-cache}
Сбрасывает внутренний DNS кеш ClickHouse. Иногда (для старых версий ClickHouse) необходимо использовать эту команду при изменении инфраструктуры (смене IP адреса у другого ClickHouse сервера или сервера, используемого словарями). Сбрасывает внутренний DNS кеш ClickHouse. Иногда (для старых версий ClickHouse) необходимо использовать эту команду при изменении инфраструктуры (смене IP адреса у другого ClickHouse сервера или сервера, используемого словарями).

View File

@ -34,6 +34,10 @@ using RWLock = std::shared_ptr<RWLockImpl>;
/// - SELECT thread 1 locks in the Read mode /// - SELECT thread 1 locks in the Read mode
/// - ALTER tries to lock in the Write mode (waits for SELECT thread 1) /// - ALTER tries to lock in the Write mode (waits for SELECT thread 1)
/// - SELECT thread 2 tries to lock in the Read mode (waits for ALTER) /// - SELECT thread 2 tries to lock in the Read mode (waits for ALTER)
///
/// NOTE: it is dangerous to acquire lock with NO_QUERY, because FastPath doesn't
/// exist for this case and deadlock, described in previous note,
/// may accur in case of recursive locking.
class RWLockImpl : public std::enable_shared_from_this<RWLockImpl> class RWLockImpl : public std::enable_shared_from_this<RWLockImpl>
{ {
public: public:

View File

@ -16,6 +16,7 @@ struct Settings;
* and should not be changed by the user without a reason. * and should not be changed by the user without a reason.
*/ */
#define LIST_OF_COORDINATION_SETTINGS(M) \ #define LIST_OF_COORDINATION_SETTINGS(M) \
M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_SESSION_TIMEOUT_MS, "Default client session timeout", 0) \ M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_SESSION_TIMEOUT_MS, "Default client session timeout", 0) \
M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \ M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \
@ -36,7 +37,8 @@ struct Settings;
M(UInt64, max_requests_batch_size, 100, "Max size of batch in requests count before it will be sent to RAFT", 0) \ M(UInt64, max_requests_batch_size, 100, "Max size of batch in requests count before it will be sent to RAFT", 0) \
M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \
M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \
M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \
M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0)
DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS)

View File

@ -10,6 +10,7 @@
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/copyData.h> #include <IO/copyData.h>
#include <filesystem> #include <filesystem>
#include <memory>
namespace DB namespace DB
{ {
@ -32,9 +33,12 @@ namespace
return parse<uint64_t>(name_parts[1]); return parse<uint64_t>(name_parts[1]);
} }
std::string getSnapshotFileName(uint64_t up_to_log_idx) std::string getSnapshotFileName(uint64_t up_to_log_idx, bool compress_zstd)
{ {
return std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin"; auto base = std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin";
if (compress_zstd)
base += ".zstd";
return base;
} }
std::string getBaseName(const String & path) std::string getBaseName(const String & path)
@ -218,6 +222,7 @@ SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage,
storage.zxid = result->get_last_log_idx(); storage.zxid = result->get_last_log_idx();
storage.session_id_counter = session_id; storage.session_id_counter = session_id;
/// Before V1 we serialized ACL without acl_map
if (current_version >= SnapshotVersion::V1) if (current_version >= SnapshotVersion::V1)
{ {
size_t acls_map_size; size_t acls_map_size;
@ -338,9 +343,13 @@ KeeperStorageSnapshot::~KeeperStorageSnapshot()
storage->disableSnapshotMode(); storage->disableSnapshotMode();
} }
KeeperSnapshotManager::KeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, const std::string & superdigest_, size_t storage_tick_time_) KeeperSnapshotManager::KeeperSnapshotManager(
const std::string & snapshots_path_, size_t snapshots_to_keep_,
bool compress_snapshots_zstd_,
const std::string & superdigest_, size_t storage_tick_time_)
: snapshots_path(snapshots_path_) : snapshots_path(snapshots_path_)
, snapshots_to_keep(snapshots_to_keep_) , snapshots_to_keep(snapshots_to_keep_)
, compress_snapshots_zstd(compress_snapshots_zstd_)
, superdigest(superdigest_) , superdigest(superdigest_)
, storage_tick_time(storage_tick_time_) , storage_tick_time(storage_tick_time_)
{ {
@ -380,7 +389,7 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer
{ {
ReadBufferFromNuraftBuffer reader(buffer); ReadBufferFromNuraftBuffer reader(buffer);
auto snapshot_file_name = getSnapshotFileName(up_to_log_idx); auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd);
auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name;
std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name; std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name;
std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name; std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name;
@ -426,22 +435,46 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFrom
return writer.getBuffer(); return writer.getBuffer();
} }
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const
{ {
WriteBufferFromNuraftBuffer writer; std::unique_ptr<WriteBufferFromNuraftBuffer> writer = std::make_unique<WriteBufferFromNuraftBuffer>();
CompressedWriteBuffer compressed_writer(writer); auto * buffer_raw_ptr = writer.get();
std::unique_ptr<WriteBuffer> compressed_writer;
if (compress_snapshots_zstd)
compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3);
else
compressed_writer = std::make_unique<CompressedWriteBuffer>(*writer);
KeeperStorageSnapshot::serialize(snapshot, compressed_writer); KeeperStorageSnapshot::serialize(snapshot, *compressed_writer);
compressed_writer.finalize(); compressed_writer->finalize();
return writer.getBuffer(); return buffer_raw_ptr->getBuffer();
}
bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer)
{
static constexpr uint32_t ZSTD_COMPRESSED_MAGIC = 0xFD2FB528;
ReadBufferFromNuraftBuffer reader(buffer);
uint32_t magic_from_buffer;
reader.readStrict(reinterpret_cast<char *>(&magic_from_buffer), sizeof(magic_from_buffer));
buffer->pos(0);
return magic_from_buffer == ZSTD_COMPRESSED_MAGIC;
} }
SnapshotMetaAndStorage KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const SnapshotMetaAndStorage KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
{ {
ReadBufferFromNuraftBuffer reader(buffer); bool is_zstd_compressed = isZstdCompressed(buffer);
CompressedReadBuffer compressed_reader(reader);
std::unique_ptr<ReadBufferFromNuraftBuffer> reader = std::make_unique<ReadBufferFromNuraftBuffer>(buffer);
std::unique_ptr<ReadBuffer> compressed_reader;
if (is_zstd_compressed)
compressed_reader = wrapReadBufferWithCompressionMethod(std::move(reader), CompressionMethod::Zstd);
else
compressed_reader = std::make_unique<CompressedReadBuffer>(*reader);
auto storage = std::make_unique<KeeperStorage>(storage_tick_time, superdigest); auto storage = std::make_unique<KeeperStorage>(storage_tick_time, superdigest);
auto snapshot_metadata = KeeperStorageSnapshot::deserialize(*storage, compressed_reader); auto snapshot_metadata = KeeperStorageSnapshot::deserialize(*storage, *compressed_reader);
return std::make_pair(snapshot_metadata, std::move(storage)); return std::make_pair(snapshot_metadata, std::move(storage));
} }

View File

@ -15,10 +15,19 @@ enum SnapshotVersion : uint8_t
V0 = 0, V0 = 0,
V1 = 1, /// with ACL map V1 = 1, /// with ACL map
V2 = 2, /// with 64 bit buffer header V2 = 2, /// with 64 bit buffer header
V3 = 3, /// compress snapshots with ZSTD codec
}; };
static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V2; static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V3;
/// In memory keeper snapshot. Keeper Storage based on a hash map which can be
/// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot
/// class do it in constructor. It also copies iterators from storage hash table
/// up to some log index with lock. In destructor this class turn off snapshot
/// mode for KeeperStorage.
///
/// This representation of snapshot have to be serialized into NuRaft
/// buffer and send over network or saved to file.
struct KeeperStorageSnapshot struct KeeperStorageSnapshot
{ {
public: public:
@ -34,12 +43,20 @@ public:
KeeperStorage * storage; KeeperStorage * storage;
SnapshotVersion version = CURRENT_SNAPSHOT_VERSION; SnapshotVersion version = CURRENT_SNAPSHOT_VERSION;
/// Snapshot metadata
SnapshotMetadataPtr snapshot_meta; SnapshotMetadataPtr snapshot_meta;
/// Max session id
int64_t session_id; int64_t session_id;
/// Size of snapshot container in amount of nodes after begin iterator
/// so we have for loop for (i = 0; i < snapshot_container_size; ++i) { doSmth(begin + i); }
size_t snapshot_container_size; size_t snapshot_container_size;
/// Iterator to the start of the storage
KeeperStorage::Container::const_iterator begin; KeeperStorage::Container::const_iterator begin;
/// Active sessions and their timeouts
SessionAndTimeout session_and_timeout; SessionAndTimeout session_and_timeout;
/// Sessions credentials
KeeperStorage::SessionAndAuth session_and_auth; KeeperStorage::SessionAndAuth session_and_auth;
/// ACLs cache for better performance. Without we cannot deserialize storage.
std::unordered_map<uint64_t, Coordination::ACLs> acl_map; std::unordered_map<uint64_t, Coordination::ACLs> acl_map;
}; };
@ -49,28 +66,42 @@ using CreateSnapshotCallback = std::function<void(KeeperStorageSnapshotPtr &&)>;
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>; using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>;
/// Class responsible for snapshots serialization and deserialization. Each snapshot
/// has it's path on disk and log index.
class KeeperSnapshotManager class KeeperSnapshotManager
{ {
public: public:
KeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, const std::string & superdigest_ = "", size_t storage_tick_time_ = 500); KeeperSnapshotManager(
const std::string & snapshots_path_, size_t snapshots_to_keep_,
bool compress_snapshots_zstd_ = true, const std::string & superdigest_ = "", size_t storage_tick_time_ = 500);
/// Restore storage from latest available snapshot
SnapshotMetaAndStorage restoreFromLatestSnapshot(); SnapshotMetaAndStorage restoreFromLatestSnapshot();
static nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot); /// Compress snapshot and serialize it to buffer
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const;
/// Serialize already compressed snapshot to disk (return path)
std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const; SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
/// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer.
nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const; nuraft::ptr<nuraft::buffer> deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const;
/// Deserialize latest snapshot from disk into compressed nuraft buffer.
nuraft::ptr<nuraft::buffer> deserializeLatestSnapshotBufferFromDisk(); nuraft::ptr<nuraft::buffer> deserializeLatestSnapshotBufferFromDisk();
/// Remove snapshot with this log_index
void removeSnapshot(uint64_t log_idx); void removeSnapshot(uint64_t log_idx);
/// Total amount of snapshots
size_t totalSnapshots() const size_t totalSnapshots() const
{ {
return existing_snapshots.size(); return existing_snapshots.size();
} }
/// The most fresh snapshot log index we have
size_t getLatestSnapshotIndex() const size_t getLatestSnapshotIndex() const
{ {
if (!existing_snapshots.empty()) if (!existing_snapshots.empty())
@ -80,13 +111,28 @@ public:
private: private:
void removeOutdatedSnapshotsIfNeeded(); void removeOutdatedSnapshotsIfNeeded();
/// Checks first 4 buffer bytes to became sure that snapshot compressed with
/// ZSTD codec.
static bool isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer);
const std::string snapshots_path; const std::string snapshots_path;
/// How many snapshots to keep before remove
const size_t snapshots_to_keep; const size_t snapshots_to_keep;
/// All existing snapshots in our path (log_index -> path)
std::map<uint64_t, std::string> existing_snapshots; std::map<uint64_t, std::string> existing_snapshots;
/// Compress snapshots in common ZSTD format instead of custom ClickHouse block LZ4 format
const bool compress_snapshots_zstd;
/// Superdigest for deserialization of storage
const std::string superdigest; const std::string superdigest;
/// Storage sessions timeout check interval (also for deserializatopn)
size_t storage_tick_time; size_t storage_tick_time;
}; };
/// Keeper create snapshots in background thread. KeeperStateMachine just create
/// in-memory snapshot from storage and push task for it serialization into
/// special tasks queue. Background thread check this queue and after snapshot
/// successfully serialized notify state machine.
struct CreateSnapshotTask struct CreateSnapshotTask
{ {
KeeperStorageSnapshotPtr snapshot; KeeperStorageSnapshotPtr snapshot;

View File

@ -46,7 +46,10 @@ KeeperStateMachine::KeeperStateMachine(
const CoordinationSettingsPtr & coordination_settings_, const CoordinationSettingsPtr & coordination_settings_,
const std::string & superdigest_) const std::string & superdigest_)
: coordination_settings(coordination_settings_) : coordination_settings(coordination_settings_)
, snapshot_manager(snapshots_path_, coordination_settings->snapshots_to_keep, superdigest_, coordination_settings->dead_session_check_period_ms.totalMicroseconds()) , snapshot_manager(
snapshots_path_, coordination_settings->snapshots_to_keep,
coordination_settings->compress_snapshots_with_zstd_format, superdigest_,
coordination_settings->dead_session_check_period_ms.totalMicroseconds())
, responses_queue(responses_queue_) , responses_queue(responses_queue_)
, snapshots_queue(snapshots_queue_) , snapshots_queue(snapshots_queue_)
, last_committed_idx(0) , last_committed_idx(0)

View File

@ -934,8 +934,9 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s
TEST_P(CoordinationTest, TestStorageSnapshotSimple) TEST_P(CoordinationTest, TestStorageSnapshotSimple)
{ {
auto params = GetParam();
ChangelogDirTest test("./snapshots"); ChangelogDirTest test("./snapshots");
DB::KeeperSnapshotManager manager("./snapshots", 3); DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression);
DB::KeeperStorage storage(500, ""); DB::KeeperStorage storage(500, "");
addNode(storage, "/hello", "world", 1); addNode(storage, "/hello", "world", 1);
@ -956,7 +957,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple)
auto buf = manager.serializeSnapshotToBuffer(snapshot); auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 2); manager.serializeSnapshotBufferToDisk(*buf, 2);
EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin")); EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension));
auto debuf = manager.deserializeSnapshotBufferFromDisk(2); auto debuf = manager.deserializeSnapshotBufferFromDisk(2);
@ -981,8 +982,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple)
TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites)
{ {
auto params = GetParam();
ChangelogDirTest test("./snapshots"); ChangelogDirTest test("./snapshots");
DB::KeeperSnapshotManager manager("./snapshots", 3); DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression);
DB::KeeperStorage storage(500, ""); DB::KeeperStorage storage(500, "");
storage.getSessionID(130); storage.getSessionID(130);
@ -1005,7 +1007,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites)
auto buf = manager.serializeSnapshotToBuffer(snapshot); auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 50); manager.serializeSnapshotBufferToDisk(*buf, 50);
EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin")); EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension));
auto debuf = manager.deserializeSnapshotBufferFromDisk(50); auto debuf = manager.deserializeSnapshotBufferFromDisk(50);
@ -1021,8 +1023,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites)
TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots)
{ {
auto params = GetParam();
ChangelogDirTest test("./snapshots"); ChangelogDirTest test("./snapshots");
DB::KeeperSnapshotManager manager("./snapshots", 3); DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression);
DB::KeeperStorage storage(500, ""); DB::KeeperStorage storage(500, "");
storage.getSessionID(130); storage.getSessionID(130);
@ -1037,14 +1040,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots)
DB::KeeperStorageSnapshot snapshot(&storage, j * 50); DB::KeeperStorageSnapshot snapshot(&storage, j * 50);
auto buf = manager.serializeSnapshotToBuffer(snapshot); auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, j * 50); manager.serializeSnapshotBufferToDisk(*buf, j * 50);
EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin")); EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin" + params.extension));
} }
EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin")); EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin" + params.extension));
EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin")); EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin" + params.extension));
EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin")); EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin" + params.extension));
EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin")); EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin" + params.extension));
EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin")); EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin" + params.extension));
auto [meta, restored_storage] = manager.restoreFromLatestSnapshot(); auto [meta, restored_storage] = manager.restoreFromLatestSnapshot();
@ -1059,8 +1062,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots)
TEST_P(CoordinationTest, TestStorageSnapshotMode) TEST_P(CoordinationTest, TestStorageSnapshotMode)
{ {
auto params = GetParam();
ChangelogDirTest test("./snapshots"); ChangelogDirTest test("./snapshots");
DB::KeeperSnapshotManager manager("./snapshots", 3); DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression);
DB::KeeperStorage storage(500, ""); DB::KeeperStorage storage(500, "");
for (size_t i = 0; i < 50; ++i) for (size_t i = 0; i < 50; ++i)
{ {
@ -1087,7 +1091,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode)
auto buf = manager.serializeSnapshotToBuffer(snapshot); auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 50); manager.serializeSnapshotBufferToDisk(*buf, 50);
} }
EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin")); EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension));
EXPECT_EQ(storage.container.size(), 26); EXPECT_EQ(storage.container.size(), 26);
storage.clearGarbageAfterSnapshot(); storage.clearGarbageAfterSnapshot();
EXPECT_EQ(storage.container.snapshotSize(), 26); EXPECT_EQ(storage.container.snapshotSize(), 26);
@ -1110,8 +1114,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode)
TEST_P(CoordinationTest, TestStorageSnapshotBroken) TEST_P(CoordinationTest, TestStorageSnapshotBroken)
{ {
auto params = GetParam();
ChangelogDirTest test("./snapshots"); ChangelogDirTest test("./snapshots");
DB::KeeperSnapshotManager manager("./snapshots", 3); DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression);
DB::KeeperStorage storage(500, ""); DB::KeeperStorage storage(500, "");
for (size_t i = 0; i < 50; ++i) for (size_t i = 0; i < 50; ++i)
{ {
@ -1122,10 +1127,10 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken)
auto buf = manager.serializeSnapshotToBuffer(snapshot); auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 50); manager.serializeSnapshotBufferToDisk(*buf, 50);
} }
EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin")); EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension));
/// Let's corrupt file /// Let's corrupt file
DB::WriteBufferFromFile plain_buf("./snapshots/snapshot_50.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); DB::WriteBufferFromFile plain_buf("./snapshots/snapshot_50.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
plain_buf.truncate(34); plain_buf.truncate(34);
plain_buf.sync(); plain_buf.sync();
@ -1464,6 +1469,52 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite)
} }
TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions)
{
auto params = GetParam();
ChangelogDirTest test("./snapshots");
DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression);
DB::KeeperStorage storage(500, "");
addNode(storage, "/hello", "world", 1);
addNode(storage, "/hello/somepath", "somedata", 3);
storage.session_id_counter = 5;
storage.zxid = 2;
storage.ephemerals[3] = {"/hello"};
storage.ephemerals[1] = {"/hello/somepath"};
storage.getSessionID(130);
storage.getSessionID(130);
DB::KeeperStorageSnapshot snapshot(&storage, 2);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 2);
EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension));
DB::KeeperSnapshotManager new_manager("./snapshots", 3, !params.enable_compression);
auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2);
auto [snapshot_meta, restored_storage] = new_manager.deserializeSnapshotFromBuffer(debuf);
EXPECT_EQ(restored_storage->container.size(), 3);
EXPECT_EQ(restored_storage->container.getValue("/").children.size(), 1);
EXPECT_EQ(restored_storage->container.getValue("/hello").children.size(), 1);
EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").children.size(), 0);
EXPECT_EQ(restored_storage->container.getValue("/").data, "");
EXPECT_EQ(restored_storage->container.getValue("/hello").data, "world");
EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").data, "somedata");
EXPECT_EQ(restored_storage->session_id_counter, 7);
EXPECT_EQ(restored_storage->zxid, 2);
EXPECT_EQ(restored_storage->ephemerals.size(), 2);
EXPECT_EQ(restored_storage->ephemerals[3].size(), 1);
EXPECT_EQ(restored_storage->ephemerals[1].size(), 1);
EXPECT_EQ(restored_storage->session_and_timeout.size(), 2);
}
INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite,
CoordinationTest, CoordinationTest,
::testing::ValuesIn(std::initializer_list<CompressionParam>{ ::testing::ValuesIn(std::initializer_list<CompressionParam>{

View File

@ -350,7 +350,7 @@ class IColumn;
M(UInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ M(UInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \
M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \
M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \
M(UInt64, memory_profiler_step, 0, "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
\ \
M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \

View File

@ -8,6 +8,7 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Common/escapeForFileName.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <common/scope_guard_safe.h> #include <common/scope_guard_safe.h>
@ -40,7 +41,7 @@ void DatabaseLazy::loadStoredObjects(
{ {
iterateMetadataFiles(local_context, [this](const String & file_name) iterateMetadataFiles(local_context, [this](const String & file_name)
{ {
const std::string table_name = file_name.substr(0, file_name.size() - 4); const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4));
fs::path detached_permanently_flag = fs::path(getMetadataPath()) / (file_name + detached_suffix); fs::path detached_permanently_flag = fs::path(getMetadataPath()) / (file_name + detached_suffix);
if (fs::exists(detached_permanently_flag)) if (fs::exists(detached_permanently_flag))

View File

@ -176,7 +176,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables
/// if (create_query->uuid != UUIDHelpers::Nil) /// if (create_query->uuid != UUIDHelpers::Nil)
/// DatabaseCatalog::instance().addUUIDMapping(create_query->uuid); /// DatabaseCatalog::instance().addUUIDMapping(create_query->uuid);
const std::string table_name = file_name.substr(0, file_name.size() - 4); const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4));
LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name));
return; return;
} }

View File

@ -603,7 +603,7 @@ namespace S3
/// Case when bucket name represented in domain name of S3 URL. /// Case when bucket name represented in domain name of S3 URL.
/// E.g. (https://bucket-name.s3.Region.amazonaws.com/key) /// E.g. (https://bucket-name.s3.Region.amazonaws.com/key)
/// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access
static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3|cos)([.\-][a-z0-9\-.:]+))"); static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3|cos|obs)([.\-][a-z0-9\-.:]+))");
/// Case when bucket name and key represented in path of S3 URL. /// Case when bucket name and key represented in path of S3 URL.
/// E.g. (https://s3.Region.amazonaws.com/bucket-name/key) /// E.g. (https://s3.Region.amazonaws.com/bucket-name/key)
@ -613,6 +613,8 @@ namespace S3
static constexpr auto S3 = "S3"; static constexpr auto S3 = "S3";
static constexpr auto COSN = "COSN"; static constexpr auto COSN = "COSN";
static constexpr auto COS = "COS"; static constexpr auto COS = "COS";
static constexpr auto OBS = "OBS";
uri = uri_; uri = uri_;
storage_name = S3; storage_name = S3;
@ -636,7 +638,7 @@ namespace S3
} }
boost::to_upper(name); boost::to_upper(name);
if (name != S3 && name != COS) if (name != S3 && name != COS && name != OBS)
{ {
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", quoteString(name)); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", quoteString(name));
} }
@ -644,6 +646,10 @@ namespace S3
{ {
storage_name = name; storage_name = name;
} }
else if (name == OBS)
{
storage_name = OBS;
}
else else
{ {
storage_name = COSN; storage_name = COSN;

View File

@ -150,7 +150,7 @@ AsynchronousInsertQueue::~AsynchronousInsertQueue()
} }
} }
void AsynchronousInsertQueue::scheduleProcessDataJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context) void AsynchronousInsertQueue::scheduleDataProcessingJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context)
{ {
/// Wrap 'unique_ptr' with 'shared_ptr' to make this /// Wrap 'unique_ptr' with 'shared_ptr' to make this
/// lambda copyable and allow to save it to the thread pool. /// lambda copyable and allow to save it to the thread pool.
@ -221,7 +221,7 @@ void AsynchronousInsertQueue::pushImpl(InsertData::EntryPtr entry, QueueIterator
data->entries.size(), data->size, queryToString(it->first.query)); data->entries.size(), data->size, queryToString(it->first.query));
if (data->size > max_data_size) if (data->size > max_data_size)
scheduleProcessDataJob(it->first, std::move(data), getContext()); scheduleDataProcessingJob(it->first, std::move(data), getContext());
} }
void AsynchronousInsertQueue::waitForProcessingQuery(const String & query_id, const Milliseconds & timeout) void AsynchronousInsertQueue::waitForProcessingQuery(const String & query_id, const Milliseconds & timeout)
@ -266,7 +266,7 @@ void AsynchronousInsertQueue::busyCheck()
auto lag = std::chrono::steady_clock::now() - elem->data->first_update; auto lag = std::chrono::steady_clock::now() - elem->data->first_update;
if (lag >= busy_timeout) if (lag >= busy_timeout)
scheduleProcessDataJob(key, std::move(elem->data), getContext()); scheduleDataProcessingJob(key, std::move(elem->data), getContext());
else else
timeout = std::min(timeout, std::chrono::ceil<std::chrono::milliseconds>(busy_timeout - lag)); timeout = std::min(timeout, std::chrono::ceil<std::chrono::milliseconds>(busy_timeout - lag));
} }
@ -288,7 +288,7 @@ void AsynchronousInsertQueue::staleCheck()
auto lag = std::chrono::steady_clock::now() - elem->data->last_update; auto lag = std::chrono::steady_clock::now() - elem->data->last_update;
if (lag >= stale_timeout) if (lag >= stale_timeout)
scheduleProcessDataJob(key, std::move(elem->data), getContext()); scheduleDataProcessingJob(key, std::move(elem->data), getContext());
} }
} }
} }
@ -369,6 +369,10 @@ try
insert_context->makeQueryContext(); insert_context->makeQueryContext();
insert_context->setSettings(key.settings); insert_context->setSettings(key.settings);
/// Set initial_query_id, because it's used in InterpreterInsertQuery for table lock.
insert_context->getClientInfo().query_kind = ClientInfo::QueryKind::INITIAL_QUERY;
insert_context->setCurrentQueryId("");
InterpreterInsertQuery interpreter(key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true); InterpreterInsertQuery interpreter(key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true);
auto pipeline = interpreter.execute().pipeline; auto pipeline = interpreter.execute().pipeline;
assert(pipeline.pushing()); assert(pipeline.pushing());
@ -431,6 +435,10 @@ catch (const Exception & e)
{ {
finishWithException(key.query, data->entries, e); finishWithException(key.query, data->entries, e);
} }
catch (const Poco::Exception & e)
{
finishWithException(key.query, data->entries, e);
}
catch (const std::exception & e) catch (const std::exception & e)
{ {
finishWithException(key.query, data->entries, e); finishWithException(key.query, data->entries, e);

View File

@ -106,13 +106,10 @@ private:
QueryIdToEntry currently_processing_queries; QueryIdToEntry currently_processing_queries;
/// Logic and events behind queue are as follows: /// Logic and events behind queue are as follows:
/// - reset_timeout: if queue is empty for some time, then we delete the queue and free all associated resources, e.g. tables.
/// - busy_timeout: if queue is active for too long and there are a lot of rapid inserts, then we dump the data, so it doesn't /// - busy_timeout: if queue is active for too long and there are a lot of rapid inserts, then we dump the data, so it doesn't
/// grow for a long period of time and users will be able to select new data in deterministic manner. /// grow for a long period of time and users will be able to select new data in deterministic manner.
/// - stale_timeout: if queue is stale for too long, then we dump the data too, so that users will be able to select the last /// - stale_timeout: if queue is stale for too long, then we dump the data too, so that users will be able to select the last
/// piece of inserted data. /// piece of inserted data.
/// - access_timeout: also we have to check if user still has access to the tables periodically, and if the access is lost, then
/// we dump pending data and delete queue immediately.
/// - max_data_size: if the maximum size of data is reached, then again we dump the data. /// - max_data_size: if the maximum size of data is reached, then again we dump the data.
const size_t max_data_size; /// in bytes const size_t max_data_size; /// in bytes
@ -134,7 +131,7 @@ private:
/// Should be called with shared or exclusively locked 'rwlock'. /// Should be called with shared or exclusively locked 'rwlock'.
void pushImpl(InsertData::EntryPtr entry, QueueIterator it); void pushImpl(InsertData::EntryPtr entry, QueueIterator it);
void scheduleProcessDataJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context); void scheduleDataProcessingJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context);
static void processData(InsertQuery key, InsertDataPtr data, ContextPtr global_context); static void processData(InsertQuery key, InsertDataPtr data, ContextPtr global_context);
template <typename E> template <typename E>

View File

@ -128,7 +128,7 @@ void DictionaryReader::readKeys(const IColumn & keys, Block & out_block, ColumnV
found.swap(typeid_cast<ColumnVector<UInt8> &>(*mutable_has).getData()); found.swap(typeid_cast<ColumnVector<UInt8> &>(*mutable_has).getData());
has_column.column = nullptr; has_column.column = nullptr;
/// set mapping form source keys to resulting rows in output block /// set mapping from source keys to resulting rows in output block
positions.clear(); positions.clear();
positions.resize(size, 0); positions.resize(size, 0);
size_t pos = 0; size_t pos = 0;

View File

@ -1056,11 +1056,12 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_
/// That's an input columns we need. /// That's an input columns we need.
NameSet required_names = required_output_; NameSet required_names = required_output_;
for (const auto & name : analyzed_join->keyNamesLeft()) for (const auto & name : analyzed_join->getAllNames(JoinTableSide::Left))
required_names.emplace(name); required_names.emplace(name);
if (ASTPtr extra_condition_column = analyzed_join->joinConditionColumn(JoinTableSide::Left)) for (const auto & onexpr : analyzed_join->getClauses())
required_names.emplace(extra_condition_column->getColumnName()); if (const auto & cond_name = onexpr.condColumnNames().first; !cond_name.empty())
required_names.emplace(cond_name);
for (const auto & column : required_columns) for (const auto & column : required_columns)
{ {

View File

@ -1482,18 +1482,15 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
const Settings & settings = context->getSettingsRef(); const Settings & settings = context->getSettingsRef();
const ConstStoragePtr & storage = query_analyzer.storage(); const ConstStoragePtr & storage = query_analyzer.storage();
bool finalized = false; ssize_t prewhere_step_num = -1;
size_t where_step_num = 0; ssize_t where_step_num = -1;
ssize_t having_step_num = -1;
auto finalize_chain = [&](ExpressionActionsChain & chain) auto finalize_chain = [&](ExpressionActionsChain & chain)
{ {
chain.finalize(); chain.finalize();
if (!finalized) finalize(chain, prewhere_step_num, where_step_num, having_step_num, query);
{
finalize(chain, where_step_num, query);
finalized = true;
}
chain.clear(); chain.clear();
}; };
@ -1524,6 +1521,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere))
{ {
/// Prewhere is always the first one.
prewhere_step_num = 0;
prewhere_info = std::make_shared<PrewhereInfo>(actions, query.prewhere()->getColumnName()); prewhere_info = std::make_shared<PrewhereInfo>(actions, query.prewhere()->getColumnName());
if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings))
@ -1592,6 +1591,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
if (query_analyzer.appendHaving(chain, only_types || !second_stage)) if (query_analyzer.appendHaving(chain, only_types || !second_stage))
{ {
having_step_num = chain.steps.size() - 1;
before_having = chain.getLastActions(); before_having = chain.getLastActions();
chain.addStep(); chain.addStep();
} }
@ -1692,13 +1692,16 @@ ExpressionAnalysisResult::ExpressionAnalysisResult(
checkActions(); checkActions();
} }
void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, size_t where_step_num, const ASTSelectQuery & query) void ExpressionAnalysisResult::finalize(
const ExpressionActionsChain & chain,
ssize_t & prewhere_step_num,
ssize_t & where_step_num,
ssize_t & having_step_num,
const ASTSelectQuery & query)
{ {
size_t next_step_i = 0; if (prewhere_step_num >= 0)
if (hasPrewhere())
{ {
const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); const ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num);
prewhere_info->prewhere_actions->projectInput(false); prewhere_info->prewhere_actions->projectInput(false);
NameSet columns_to_remove; NameSet columns_to_remove;
@ -1711,12 +1714,21 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si
} }
columns_to_remove_after_prewhere = std::move(columns_to_remove); columns_to_remove_after_prewhere = std::move(columns_to_remove);
prewhere_step_num = -1;
} }
if (hasWhere()) if (where_step_num >= 0)
{ {
where_column_name = query.where()->getColumnName(); where_column_name = query.where()->getColumnName();
remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second; remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second;
where_step_num = -1;
}
if (having_step_num >= 0)
{
having_column_name = query.having()->getColumnName();
remove_having_filter = chain.steps.at(having_step_num)->required_output.find(having_column_name)->second;
having_step_num = -1;
} }
} }

View File

@ -229,6 +229,8 @@ struct ExpressionAnalysisResult
ActionsDAGPtr before_where; ActionsDAGPtr before_where;
ActionsDAGPtr before_aggregation; ActionsDAGPtr before_aggregation;
ActionsDAGPtr before_having; ActionsDAGPtr before_having;
String having_column_name;
bool remove_having_filter = false;
ActionsDAGPtr before_window; ActionsDAGPtr before_window;
ActionsDAGPtr before_order_by; ActionsDAGPtr before_order_by;
ActionsDAGPtr before_limit_by; ActionsDAGPtr before_limit_by;
@ -274,7 +276,12 @@ struct ExpressionAnalysisResult
void removeExtraColumns() const; void removeExtraColumns() const;
void checkActions() const; void checkActions() const;
void finalize(const ExpressionActionsChain & chain, size_t where_step_num, const ASTSelectQuery & query); void finalize(
const ExpressionActionsChain & chain,
ssize_t & prewhere_step_num,
ssize_t & where_step_num,
ssize_t & having_step_num,
const ASTSelectQuery & query);
}; };
/// SelectQuery specific ExpressionAnalyzer part. /// SelectQuery specific ExpressionAnalyzer part.

File diff suppressed because it is too large Load Diff

View File

@ -4,6 +4,7 @@
#include <optional> #include <optional>
#include <shared_mutex> #include <shared_mutex>
#include <deque> #include <deque>
#include <vector>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
@ -35,27 +36,36 @@ namespace JoinStuff
/// Flags needed to implement RIGHT and FULL JOINs. /// Flags needed to implement RIGHT and FULL JOINs.
class JoinUsedFlags class JoinUsedFlags
{ {
std::vector<std::atomic_bool> flags; using RawBlockPtr = const Block *;
using UsedFlagsForBlock = std::vector<std::atomic_bool>;
/// For multiple dijuncts each empty in hashmap stores flags for particular block
/// For single dicunct we store all flags in `nullptr` entry, index is the offset in FindResult
std::unordered_map<RawBlockPtr, UsedFlagsForBlock> flags;
bool need_flags; bool need_flags;
public: public:
/// Update size for vector with flags. /// Update size for vector with flags.
/// Calling this method invalidates existing flags. /// Calling this method invalidates existing flags.
/// It can be called several times, but all of them should happen before using this structure. /// It can be called several times, but all of them should happen before using this structure.
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS> template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
void reinit(size_t size_); void reinit(size_t size_);
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
void reinit(const Block * block_ptr);
bool getUsedSafe(size_t i) const; bool getUsedSafe(size_t i) const;
bool getUsedSafe(const Block * block_ptr, size_t row_idx) const;
template <bool use_flags> template <bool use_flags, bool multiple_disjuncts, typename T>
void setUsed(size_t i); void setUsed(const T & f);
template <bool use_flags> template <bool use_flags, bool multiple_disjuncts, typename T>
bool getUsed(size_t i); bool getUsed(const T & f);
template <bool use_flags> template <bool use_flags, bool multiple_disjuncts, typename T>
bool setUsedOnce(size_t i); bool setUsedOnce(const T & f);
}; };
} }
@ -165,7 +175,8 @@ public:
* Use only after all calls to joinBlock was done. * Use only after all calls to joinBlock was done.
* left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside). * left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside).
*/ */
std::shared_ptr<NotJoinedBlocks> getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const override; std::shared_ptr<NotJoinedBlocks> getNonJoinedBlocks(
const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
/// Number of keys in all built JOIN maps. /// Number of keys in all built JOIN maps.
size_t getTotalRowCount() const final; size_t getTotalRowCount() const final;
@ -180,11 +191,7 @@ public:
ASOF::Inequality getAsofInequality() const { return asof_inequality; } ASOF::Inequality getAsofInequality() const { return asof_inequality; }
bool anyTakeLastRow() const { return any_take_last_row; } bool anyTakeLastRow() const { return any_take_last_row; }
const ColumnWithTypeAndName & rightAsofKeyColumn() const const ColumnWithTypeAndName & rightAsofKeyColumn() const;
{
/// It should be nullable if nullable_right_side is true
return savedBlockSample().getByName(key_names_right.back());
}
/// Different types of keys for maps. /// Different types of keys for maps.
#define APPLY_FOR_JOIN_VARIANTS(M) \ #define APPLY_FOR_JOIN_VARIANTS(M) \
@ -305,14 +312,16 @@ public:
using MapsAsof = MapsTemplate<AsofRowRefs>; using MapsAsof = MapsTemplate<AsofRowRefs>;
using MapsVariant = std::variant<MapsOne, MapsAll, MapsAsof>; using MapsVariant = std::variant<MapsOne, MapsAll, MapsAsof>;
using BlockNullmapList = std::deque<std::pair<const Block *, ColumnPtr>>;
using RawBlockPtr = const Block *;
using BlockNullmapList = std::deque<std::pair<RawBlockPtr, ColumnPtr>>;
struct RightTableData struct RightTableData
{ {
Type type = Type::EMPTY; Type type = Type::EMPTY;
bool empty = true; bool empty = true;
MapsVariant maps; std::vector<MapsVariant> maps;
Block sample_block; /// Block as it would appear in the BlockList Block sample_block; /// Block as it would appear in the BlockList
BlocksList blocks; /// Blocks of "right" table. BlocksList blocks; /// Blocks of "right" table.
BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed) BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed)
@ -321,6 +330,8 @@ public:
Arena pool; Arena pool;
}; };
using RightTableDataPtr = std::shared_ptr<RightTableData>;
/// We keep correspondence between used_flags and hash table internal buffer. /// We keep correspondence between used_flags and hash table internal buffer.
/// Hash table cannot be modified during HashJoin lifetime and must be protected with lock. /// Hash table cannot be modified during HashJoin lifetime and must be protected with lock.
void setLock(std::shared_mutex & rwlock) void setLock(std::shared_mutex & rwlock)
@ -330,15 +341,14 @@ public:
void reuseJoinedData(const HashJoin & join); void reuseJoinedData(const HashJoin & join);
std::shared_ptr<RightTableData> getJoinedData() const RightTableDataPtr getJoinedData() const { return data; }
{
return data;
}
bool isUsed(size_t off) const { return used_flags.getUsedSafe(off); } bool isUsed(size_t off) const { return used_flags.getUsedSafe(off); }
bool isUsed(const Block * block_ptr, size_t row_idx) const { return used_flags.getUsedSafe(block_ptr, row_idx); }
private: private:
friend class NotJoinedHash; template<bool> friend class NotJoinedHash;
friend class JoinSource; friend class JoinSource;
std::shared_ptr<TableJoin> table_join; std::shared_ptr<TableJoin> table_join;
@ -348,9 +358,6 @@ private:
/// This join was created from StorageJoin and it is already filled. /// This join was created from StorageJoin and it is already filled.
bool from_storage_join = false; bool from_storage_join = false;
/// Names of key columns in right-side table (in the order they appear in ON/USING clause). @note It could contain duplicates.
const Names & key_names_right;
bool nullable_right_side; /// In case of LEFT and FULL joins, if use_nulls, convert right-side columns to Nullable. bool nullable_right_side; /// In case of LEFT and FULL joins, if use_nulls, convert right-side columns to Nullable.
bool nullable_left_side; /// In case of RIGHT and FULL joins, if use_nulls, convert left-side columns to Nullable. bool nullable_left_side; /// In case of RIGHT and FULL joins, if use_nulls, convert left-side columns to Nullable.
bool any_take_last_row; /// Overwrite existing values when encountering the same key again bool any_take_last_row; /// Overwrite existing values when encountering the same key again
@ -358,14 +365,14 @@ private:
ASOF::Inequality asof_inequality; ASOF::Inequality asof_inequality;
/// Right table data. StorageJoin shares it between many Join objects. /// Right table data. StorageJoin shares it between many Join objects.
std::shared_ptr<RightTableData> data;
/// Flags that indicate that particular row already used in join. /// Flags that indicate that particular row already used in join.
/// Flag is stored for every record in hash map. /// Flag is stored for every record in hash map.
/// Number of this flags equals to hashtable buffer size (plus one for zero value). /// Number of this flags equals to hashtable buffer size (plus one for zero value).
/// Changes in hash table broke correspondence, /// Changes in hash table broke correspondence,
/// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock) /// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock)
mutable JoinStuff::JoinUsedFlags used_flags; mutable JoinStuff::JoinUsedFlags used_flags;
Sizes key_sizes; RightTableDataPtr data;
std::vector<Sizes> key_sizes;
/// Block with columns from the right-side table. /// Block with columns from the right-side table.
Block right_sample_block; Block right_sample_block;
@ -378,10 +385,6 @@ private:
/// Left table column names that are sources for required_right_keys columns /// Left table column names that are sources for required_right_keys columns
std::vector<String> required_right_keys_sources; std::vector<String> required_right_keys_sources;
/// Additional conditions for rows to join from JOIN ON section
String condition_mask_column_name_left;
String condition_mask_column_name_right;
Poco::Logger * log; Poco::Logger * log;
Block totals; Block totals;
@ -390,7 +393,7 @@ private:
/// If set HashJoin instance is not available for modification (addJoinedBlock) /// If set HashJoin instance is not available for modification (addJoinedBlock)
std::shared_lock<std::shared_mutex> storage_join_lock; std::shared_lock<std::shared_mutex> storage_join_lock;
void init(Type type_); void dataMapInit(MapsVariant &);
const Block & savedBlockSample() const { return data->sample_block; } const Block & savedBlockSample() const { return data->sample_block; }
@ -401,9 +404,8 @@ private:
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps> template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
void joinBlockImpl( void joinBlockImpl(
Block & block, Block & block,
const Names & key_names_left,
const Block & block_with_columns_to_add, const Block & block_with_columns_to_add,
const Maps & maps, const std::vector<const Maps *> & maps_,
bool is_join_get = false) const; bool is_join_get = false) const;
void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const;

View File

@ -45,7 +45,8 @@ public:
/// Different query plan is used for such joins. /// Different query plan is used for such joins.
virtual bool isFilled() const { return false; } virtual bool isFilled() const { return false; }
virtual std::shared_ptr<NotJoinedBlocks> getNonJoinedBlocks(const Block &, UInt64) const = 0; virtual std::shared_ptr<NotJoinedBlocks>
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
}; };
using JoinPtr = std::shared_ptr<IJoin>; using JoinPtr = std::shared_ptr<IJoin>;

View File

@ -1248,7 +1248,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
{ {
bool final = !query.group_by_with_rollup && !query.group_by_with_cube; bool final = !query.group_by_with_rollup && !query.group_by_with_cube;
executeTotalsAndHaving( executeTotalsAndHaving(
query_plan, expressions.hasHaving(), expressions.before_having, aggregate_overflow_row, final); query_plan, expressions.hasHaving(), expressions.before_having, expressions.remove_having_filter, aggregate_overflow_row, final);
} }
if (query.group_by_with_rollup) if (query.group_by_with_rollup)
@ -1262,11 +1262,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu
throw Exception( throw Exception(
"WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING", "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING",
ErrorCodes::NOT_IMPLEMENTED); ErrorCodes::NOT_IMPLEMENTED);
executeHaving(query_plan, expressions.before_having); executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter);
} }
} }
else if (expressions.hasHaving()) else if (expressions.hasHaving())
executeHaving(query_plan, expressions.before_having); executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter);
} }
else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube) else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube)
throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::NOT_IMPLEMENTED); throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::NOT_IMPLEMENTED);
@ -2133,10 +2133,10 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
} }
void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression) void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter)
{ {
auto having_step auto having_step
= std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), false); = std::make_unique<FilterStep>(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), remove_filter);
having_step->setStepDescription("HAVING"); having_step->setStepDescription("HAVING");
query_plan.addStep(std::move(having_step)); query_plan.addStep(std::move(having_step));
@ -2144,7 +2144,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions
void InterpreterSelectQuery::executeTotalsAndHaving( void InterpreterSelectQuery::executeTotalsAndHaving(
QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final) QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final)
{ {
const Settings & settings = context->getSettingsRef(); const Settings & settings = context->getSettingsRef();
@ -2153,6 +2153,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(
overflow_row, overflow_row,
expression, expression,
has_having ? getSelectQuery().having()->getColumnName() : "", has_having ? getSelectQuery().having()->getColumnName() : "",
remove_filter,
settings.totals_mode, settings.totals_mode,
settings.totals_auto_threshold, settings.totals_auto_threshold,
final); final);

View File

@ -131,8 +131,8 @@ private:
void executeAggregation( void executeAggregation(
QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info);
void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final);
void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final); void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final);
void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression); void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter);
static void executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description); static void executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description);
/// FIXME should go through ActionsDAG to behave as a proper function /// FIXME should go through ActionsDAG to behave as a proper function
void executeWindow(QueryPlan & query_plan); void executeWindow(QueryPlan & query_plan);

View File

@ -61,9 +61,10 @@ public:
return join->alwaysReturnsEmptySet(); return join->alwaysReturnsEmptySet();
} }
std::shared_ptr<NotJoinedBlocks> getNonJoinedBlocks(const Block & block, UInt64 max_block_size) const override std::shared_ptr<NotJoinedBlocks>
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override
{ {
return join->getNonJoinedBlocks(block, max_block_size); return join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size);
} }
private: private:

View File

@ -507,7 +507,11 @@ MergeJoin::MergeJoin(std::shared_ptr<TableJoin> table_join_, const Block & right
ErrorCodes::PARAMETER_OUT_OF_BOUND); ErrorCodes::PARAMETER_OUT_OF_BOUND);
} }
std::tie(mask_column_name_left, mask_column_name_right) = table_join->joinConditionColumnNames(); if (!table_join->oneDisjunct())
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoin does not support OR in JOIN ON section");
const auto & onexpr = table_join->getOnlyClause();
std::tie(mask_column_name_left, mask_column_name_right) = onexpr.condColumnNames();
/// Add auxiliary joining keys to join only rows where conditions from JOIN ON sections holds /// Add auxiliary joining keys to join only rows where conditions from JOIN ON sections holds
/// Input boolean column converted to nullable and only rows with non NULLS value will be joined /// Input boolean column converted to nullable and only rows with non NULLS value will be joined
@ -519,8 +523,8 @@ MergeJoin::MergeJoin(std::shared_ptr<TableJoin> table_join_, const Block & right
key_names_right.push_back(deriveTempName(mask_column_name_right)); key_names_right.push_back(deriveTempName(mask_column_name_right));
} }
key_names_left.insert(key_names_left.end(), table_join->keyNamesLeft().begin(), table_join->keyNamesLeft().end()); key_names_left.insert(key_names_left.end(), onexpr.key_names_left.begin(), onexpr.key_names_left.end());
key_names_right.insert(key_names_right.end(), table_join->keyNamesRight().begin(), table_join->keyNamesRight().end()); key_names_right.insert(key_names_right.end(), onexpr.key_names_right.begin(), onexpr.key_names_right.end());
addConditionJoinColumn(right_sample_block, JoinTableSide::Right); addConditionJoinColumn(right_sample_block, JoinTableSide::Right);
JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, right_columns_to_add); JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, right_columns_to_add);
@ -654,7 +658,7 @@ bool MergeJoin::saveRightBlock(Block && block)
Block MergeJoin::modifyRightBlock(const Block & src_block) const Block MergeJoin::modifyRightBlock(const Block & src_block) const
{ {
Block block = materializeBlock(src_block); Block block = materializeBlock(src_block);
JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesRight()); JoinCommon::removeLowCardinalityInplace(block, table_join->getOnlyClause().key_names_right);
return block; return block;
} }
@ -670,7 +674,8 @@ bool MergeJoin::addJoinedBlock(const Block & src_block, bool)
void MergeJoin::checkTypesOfKeys(const Block & block) const void MergeJoin::checkTypesOfKeys(const Block & block) const
{ {
/// Do not check auxailary column for extra conditions, use original key names /// Do not check auxailary column for extra conditions, use original key names
JoinCommon::checkTypesOfKeys(block, table_join->keyNamesLeft(), right_table_keys, table_join->keyNamesRight()); const auto & onexpr = table_join->getOnlyClause();
JoinCommon::checkTypesOfKeys(block, onexpr.key_names_left, right_table_keys, onexpr.key_names_right);
} }
void MergeJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) void MergeJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
@ -1098,11 +1103,13 @@ private:
}; };
std::shared_ptr<NotJoinedBlocks> MergeJoin::getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const std::shared_ptr<NotJoinedBlocks> MergeJoin::getNonJoinedBlocks(
const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const
{ {
if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full)) if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full))
{ {
size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); size_t left_columns_count = left_sample_block.columns();
assert(left_columns_count == result_sample_block.columns() - right_columns_to_add.columns());
auto non_joined = std::make_unique<NotJoinedMerge>(*this, max_block_size); auto non_joined = std::make_unique<NotJoinedMerge>(*this, max_block_size);
return std::make_shared<NotJoinedBlocks>(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); return std::make_shared<NotJoinedBlocks>(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap());
} }

View File

@ -36,7 +36,7 @@ public:
/// Has to be called only after setTotals()/mergeRightBlocks() /// Has to be called only after setTotals()/mergeRightBlocks()
bool alwaysReturnsEmptySet() const override { return (is_right || is_inner) && min_max_right_blocks.empty(); } bool alwaysReturnsEmptySet() const override { return (is_right || is_inner) && min_max_right_blocks.empty(); }
std::shared_ptr<NotJoinedBlocks> getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const override; std::shared_ptr<NotJoinedBlocks> getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
private: private:
friend class NotJoinedMerge; friend class NotJoinedMerge;

View File

@ -77,6 +77,13 @@ struct RowRefList : RowRef
return &batch->row_refs[position]; return &batch->row_refs[position];
} }
const RowRef * operator * () const
{
if (first)
return root;
return &batch->row_refs[position];
}
void operator ++ () void operator ++ ()
{ {
if (first) if (first)

View File

@ -1,6 +1,7 @@
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
#include <Common/Exception.h>
#include <common/types.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Core/Block.h> #include <Core/Block.h>
@ -23,6 +24,7 @@
#include <Storages/StorageJoin.h> #include <Storages/StorageJoin.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <algorithm>
namespace DB namespace DB
@ -32,6 +34,7 @@ namespace ErrorCodes
{ {
extern const int TYPE_MISMATCH; extern const int TYPE_MISMATCH;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
} }
namespace namespace
@ -51,6 +54,45 @@ std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJo
} }
namespace
{
struct BothSidesTag {};
struct LeftSideTag {};
struct RightSideTag {};
template <typename SideTag = BothSidesTag, typename OnExpr, typename Func>
bool forAllKeys(OnExpr & expressions, Func callback)
{
static_assert(std::is_same_v<SideTag, BothSidesTag> ||
std::is_same_v<SideTag, LeftSideTag> ||
std::is_same_v<SideTag, RightSideTag>);
for (auto & expr : expressions)
{
if constexpr (std::is_same_v<SideTag, BothSidesTag>)
assert(expr.key_names_left.size() == expr.key_names_right.size());
size_t sz = !std::is_same_v<SideTag, RightSideTag> ? expr.key_names_left.size() : expr.key_names_right.size();
for (size_t i = 0; i < sz; ++i)
{
bool cont;
if constexpr (std::is_same_v<SideTag, BothSidesTag>)
cont = callback(expr.key_names_left[i], expr.key_names_right[i]);
if constexpr (std::is_same_v<SideTag, LeftSideTag>)
cont = callback(expr.key_names_left[i]);
if constexpr (std::is_same_v<SideTag, RightSideTag>)
cont = callback(expr.key_names_right[i]);
if (!cont)
return false;
}
}
return true;
}
}
TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_)
: size_limits(SizeLimits{settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode}) : size_limits(SizeLimits{settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode})
, default_max_bytes(settings.default_max_bytes_in_join) , default_max_bytes(settings.default_max_bytes_in_join)
@ -68,10 +110,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_)
void TableJoin::resetCollected() void TableJoin::resetCollected()
{ {
key_names_left.clear(); clauses.clear();
key_names_right.clear();
key_asts_left.clear();
key_asts_right.clear();
columns_from_joined_table.clear(); columns_from_joined_table.clear();
columns_added_by_join.clear(); columns_added_by_join.clear();
original_names.clear(); original_names.clear();
@ -82,24 +121,20 @@ void TableJoin::resetCollected()
void TableJoin::addUsingKey(const ASTPtr & ast) void TableJoin::addUsingKey(const ASTPtr & ast)
{ {
key_names_left.push_back(ast->getColumnName()); addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast);
key_names_right.push_back(ast->getAliasOrColumnName()); }
key_asts_left.push_back(ast); void TableJoin::addDisjunct()
key_asts_right.push_back(ast); {
clauses.emplace_back();
auto & right_key = key_names_right.back(); if (getStorageJoin() && clauses.size() > 1)
if (renames.count(right_key)) throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED);
right_key = renames[right_key];
} }
void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
{ {
key_names_left.push_back(left_table_ast->getColumnName()); addKey(left_table_ast->getColumnName(), right_table_ast->getAliasOrColumnName(), left_table_ast, right_table_ast);
key_names_right.push_back(right_table_ast->getAliasOrColumnName());
key_asts_left.push_back(left_table_ast);
key_asts_right.push_back(right_table_ast);
} }
/// @return how many times right key appears in ON section. /// @return how many times right key appears in ON section.
@ -109,9 +144,8 @@ size_t TableJoin::rightKeyInclusion(const String & name) const
return 0; return 0;
size_t count = 0; size_t count = 0;
for (const auto & key_name : key_names_right) for (const auto & clause : clauses)
if (name == key_name) count += std::count(clause.key_names_right.begin(), clause.key_names_right.end(), name);
++count;
return count; return count;
} }
@ -159,23 +193,33 @@ ASTPtr TableJoin::leftKeysList() const
{ {
ASTPtr keys_list = std::make_shared<ASTExpressionList>(); ASTPtr keys_list = std::make_shared<ASTExpressionList>();
keys_list->children = key_asts_left; keys_list->children = key_asts_left;
if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Left))
keys_list->children.push_back(extra_cond); for (const auto & clause : clauses)
{
if (clause.on_filter_condition_left)
keys_list->children.push_back(clause.on_filter_condition_left);
}
return keys_list; return keys_list;
} }
ASTPtr TableJoin::rightKeysList() const ASTPtr TableJoin::rightKeysList() const
{ {
ASTPtr keys_list = std::make_shared<ASTExpressionList>(); ASTPtr keys_list = std::make_shared<ASTExpressionList>();
if (hasOn()) if (hasOn())
keys_list->children = key_asts_right; keys_list->children = key_asts_right;
if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Right))
keys_list->children.push_back(extra_cond); for (const auto & clause : clauses)
{
if (clause.on_filter_condition_right)
keys_list->children.push_back(clause.on_filter_condition_right);
}
return keys_list; return keys_list;
} }
Names TableJoin::requiredJoinedNames() const Names TableJoin::requiredJoinedNames() const
{ {
Names key_names_right = getAllNames(JoinTableSide::Right);
NameSet required_columns_set(key_names_right.begin(), key_names_right.end()); NameSet required_columns_set(key_names_right.begin(), key_names_right.end());
for (const auto & joined_column : columns_added_by_join) for (const auto & joined_column : columns_added_by_join)
required_columns_set.insert(joined_column.name); required_columns_set.insert(joined_column.name);
@ -186,13 +230,14 @@ Names TableJoin::requiredJoinedNames() const
NameSet TableJoin::requiredRightKeys() const NameSet TableJoin::requiredRightKeys() const
{ {
NameSet required; NameSet required;
for (const auto & name : key_names_right) forAllKeys<RightSideTag>(clauses, [this, &required](const auto & name)
{ {
auto rename = renamedRightColumnName(name); auto rename = renamedRightColumnName(name);
for (const auto & column : columns_added_by_join) for (const auto & column : columns_added_by_join)
if (rename == column.name) if (rename == column.name)
required.insert(name); required.insert(name);
} return true;
});
return required; return required;
} }
@ -209,23 +254,21 @@ NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names
Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vector<String> & keys_sources) const Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vector<String> & keys_sources) const
{ {
const Names & left_keys = keyNamesLeft();
const Names & right_keys = keyNamesRight();
NameSet required_keys = requiredRightKeys(); NameSet required_keys = requiredRightKeys();
Block required_right_keys; Block required_right_keys;
if (required_keys.empty())
return required_right_keys;
for (size_t i = 0; i < right_keys.size(); ++i) forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name)
{ {
const String & right_key_name = right_keys[i];
if (required_keys.count(right_key_name) && !required_right_keys.has(right_key_name)) if (required_keys.count(right_key_name) && !required_right_keys.has(right_key_name))
{ {
const auto & right_key = right_table_keys.getByName(right_key_name); const auto & right_key = right_table_keys.getByName(right_key_name);
required_right_keys.insert(right_key); required_right_keys.insert(right_key);
keys_sources.push_back(left_keys[i]); keys_sources.push_back(left_key_name);
} }
} return true;
});
return required_right_keys; return required_right_keys;
} }
@ -307,6 +350,13 @@ bool TableJoin::sameStrictnessAndKind(ASTTableJoin::Strictness strictness_, ASTT
return false; return false;
} }
bool TableJoin::oneDisjunct() const
{
if (!isComma(kind()) && !isCross(kind()))
assert(!clauses.empty());
return clauses.size() <= 1;
}
bool TableJoin::allowMergeJoin() const bool TableJoin::allowMergeJoin() const
{ {
bool is_any = (strictness() == ASTTableJoin::Strictness::Any); bool is_any = (strictness() == ASTTableJoin::Strictness::Any);
@ -315,7 +365,8 @@ bool TableJoin::allowMergeJoin() const
bool all_join = is_all && (isInner(kind()) || isLeft(kind()) || isRight(kind()) || isFull(kind())); bool all_join = is_all && (isInner(kind()) || isLeft(kind()) || isRight(kind()) || isFull(kind()));
bool special_left = isLeft(kind()) && (is_any || is_semi); bool special_left = isLeft(kind()) && (is_any || is_semi);
return all_join || special_left;
return (all_join || special_left) && oneDisjunct();
} }
bool TableJoin::needStreamWithNonJoinedRows() const bool TableJoin::needStreamWithNonJoinedRows() const
@ -351,12 +402,13 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context)
if (!allowed_inner && !allowed_left) if (!allowed_inner && !allowed_left)
return false; return false;
const Names & right_keys = keyNamesRight(); if (clauses.size() != 1 || clauses[0].key_names_right.size() != 1)
if (right_keys.size() != 1)
return false; return false;
const auto & right_key = getOnlyClause().key_names_right[0];
/// TODO: support 'JOIN ... ON expr(dict_key) = table_key' /// TODO: support 'JOIN ... ON expr(dict_key) = table_key'
auto it_key = original_names.find(right_keys[0]); auto it_key = original_names.find(right_key);
if (it_key == original_names.end()) if (it_key == original_names.end())
return false; return false;
@ -373,7 +425,7 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context)
NamesAndTypesList dst_columns; NamesAndTypesList dst_columns;
for (const auto & col : sample_block) for (const auto & col : sample_block)
{ {
if (col.name == right_keys[0]) if (col.name == right_key)
continue; /// do not extract key column continue; /// do not extract key column
auto it = original_names.find(col.name); auto it = original_names.find(col.name);
@ -389,13 +441,28 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context)
return true; return true;
} }
static void renameIfNeeded(String & name, const NameToNameMap & renames)
{
if (const auto it = renames.find(name); it != renames.end())
name = it->second;
}
std::pair<ActionsDAGPtr, ActionsDAGPtr> std::pair<ActionsDAGPtr, ActionsDAGPtr>
TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns) TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns)
{ {
inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage()); inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage());
auto left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, key_names_left); NameToNameMap left_key_column_rename;
auto right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, key_names_right); NameToNameMap right_key_column_rename;
auto left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, left_key_column_rename);
auto right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, right_key_column_rename);
forAllKeys(clauses, [&](auto & left_key, auto & right_key)
{
renameIfNeeded(left_key, left_key_column_rename);
renameIfNeeded(right_key, right_key_column_rename);
return true;
});
return {left_converting_actions, right_converting_actions}; return {left_converting_actions, right_converting_actions};
} }
@ -414,10 +481,11 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
for (const auto & col : right) for (const auto & col : right)
right_types[renamedRightColumnName(col.name)] = col.type; right_types[renamedRightColumnName(col.name)] = col.type;
for (size_t i = 0; i < key_names_left.size(); ++i)
forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name)
{ {
auto ltype = left_types.find(key_names_left[i]); auto ltype = left_types.find(left_key_name);
auto rtype = right_types.find(key_names_right[i]); auto rtype = right_types.find(right_key_name);
if (ltype == left_types.end() || rtype == right_types.end()) if (ltype == left_types.end() || rtype == right_types.end())
{ {
/// Name mismatch, give up /// Name mismatch, give up
@ -427,7 +495,7 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
} }
if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second))
continue; return true;
DataTypePtr common_type; DataTypePtr common_type;
try try
@ -439,19 +507,20 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
{ {
throw DB::Exception(ErrorCodes::TYPE_MISMATCH, throw DB::Exception(ErrorCodes::TYPE_MISMATCH,
"Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}", "Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}",
key_names_left[i], ltype->second->getName(), left_key_name, ltype->second->getName(),
key_names_right[i], rtype->second->getName(), right_key_name, rtype->second->getName(),
ex.message()); ex.message());
} }
if (!allow_right && !common_type->equals(*rtype->second)) if (!allow_right && !common_type->equals(*rtype->second))
{ {
throw DB::Exception(ErrorCodes::TYPE_MISMATCH, throw DB::Exception(ErrorCodes::TYPE_MISMATCH,
"Can't change type for right table: {}: {} -> {}.", "Can't change type for right table: {}: {} -> {}.",
key_names_right[i], rtype->second->getName(), common_type->getName()); right_key_name, rtype->second->getName(), common_type->getName());
} }
left_type_map[key_names_left[i]] = right_type_map[key_names_right[i]] = common_type; left_type_map[left_key_name] = right_type_map[right_key_name] = common_type;
}
return true;
});
if (!left_type_map.empty() || !right_type_map.empty()) if (!left_type_map.empty() || !right_type_map.empty())
{ {
@ -466,7 +535,7 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig
} }
ActionsDAGPtr TableJoin::applyKeyConvertToTable( ActionsDAGPtr TableJoin::applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, Names & names_to_rename) const const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NameToNameMap & key_column_rename) const
{ {
bool has_some_to_do = false; bool has_some_to_do = false;
@ -483,17 +552,10 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable(
if (!has_some_to_do) if (!has_some_to_do)
return nullptr; return nullptr;
NameToNameMap key_column_rename;
/// Returns converting actions for tables that need to be performed before join /// Returns converting actions for tables that need to be performed before join
auto dag = ActionsDAG::makeConvertingActions( auto dag = ActionsDAG::makeConvertingActions(
cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, !hasUsing(), &key_column_rename); cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, !hasUsing(), &key_column_rename);
for (auto & name : names_to_rename)
{
const auto it = key_column_rename.find(name);
if (it != key_column_rename.end())
name = it->second;
}
return dag; return dag;
} }
@ -518,14 +580,34 @@ String TableJoin::renamedRightColumnName(const String & name) const
return name; return name;
} }
void TableJoin::addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast)
{
clauses.back().key_names_left.emplace_back(left_name);
key_asts_left.emplace_back(left_ast);
clauses.back().key_names_right.emplace_back(right_name);
key_asts_right.emplace_back(right_ast ? right_ast : left_ast);
}
static void addJoinConditionWithAnd(ASTPtr & current_cond, const ASTPtr & new_cond)
{
if (current_cond == nullptr)
/// no conditions, set new one
current_cond = new_cond;
else if (const auto * func = current_cond->as<ASTFunction>(); func && func->name == "and")
/// already have `and` in condition, just add new argument
func->arguments->children.push_back(new_cond);
else
/// already have some conditions, unite it with `and`
current_cond = makeASTFunction("and", current_cond, new_cond);
}
void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left)
{ {
LOG_TRACE(&Poco::Logger::get("TableJoin"), "Add join condition for {} table: {}", (is_left ? "left" : "right"), queryToString(ast)); auto & cond_ast = is_left ? clauses.back().on_filter_condition_left : clauses.back().on_filter_condition_right;
LOG_TRACE(&Poco::Logger::get("TableJoin"), "Adding join condition for {} table: {} -> {}",
if (is_left) (is_left ? "left" : "right"), ast ? queryToString(ast) : "NULL", cond_ast ? queryToString(cond_ast) : "NULL");
on_filter_condition_asts_left.push_back(ast); addJoinConditionWithAnd(cond_ast, ast);
else
on_filter_condition_asts_right.push_back(ast);
} }
std::unordered_map<String, String> TableJoin::leftToRightKeyRemap() const std::unordered_map<String, String> TableJoin::leftToRightKeyRemap() const
@ -534,50 +616,38 @@ std::unordered_map<String, String> TableJoin::leftToRightKeyRemap() const
if (hasUsing()) if (hasUsing())
{ {
const auto & required_right_keys = requiredRightKeys(); const auto & required_right_keys = requiredRightKeys();
for (size_t i = 0; i < key_names_left.size(); ++i) forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name)
{ {
const String & left_key_name = key_names_left[i];
const String & right_key_name = key_names_right[i];
if (!required_right_keys.contains(right_key_name)) if (!required_right_keys.contains(right_key_name))
left_to_right_key_remap[left_key_name] = right_key_name; left_to_right_key_remap[left_key_name] = right_key_name;
} return true;
});
} }
return left_to_right_key_remap; return left_to_right_key_remap;
} }
/// Returns all conditions related to one table joined with 'and' function Names TableJoin::getAllNames(JoinTableSide side) const
static ASTPtr buildJoinConditionColumn(const ASTs & on_filter_condition_asts)
{
if (on_filter_condition_asts.empty())
return nullptr;
if (on_filter_condition_asts.size() == 1)
return on_filter_condition_asts[0];
auto function = std::make_shared<ASTFunction>();
function->name = "and";
function->arguments = std::make_shared<ASTExpressionList>();
function->children.push_back(function->arguments);
function->arguments->children = on_filter_condition_asts;
return function;
}
ASTPtr TableJoin::joinConditionColumn(JoinTableSide side) const
{ {
Names res;
auto func = [&res](const auto & name) { res.emplace_back(name); return true; };
if (side == JoinTableSide::Left) if (side == JoinTableSide::Left)
return buildJoinConditionColumn(on_filter_condition_asts_left); forAllKeys<LeftSideTag>(clauses, func);
return buildJoinConditionColumn(on_filter_condition_asts_right); else
} forAllKeys<RightSideTag>(clauses, func);
std::pair<String, String> TableJoin::joinConditionColumnNames() const
{
std::pair<String, String> res;
if (auto cond_ast = joinConditionColumn(JoinTableSide::Left))
res.first = cond_ast->getColumnName();
if (auto cond_ast = joinConditionColumn(JoinTableSide::Right))
res.second = cond_ast->getColumnName();
return res; return res;
} }
void TableJoin::assertHasOneOnExpr() const
{
if (!oneDisjunct())
{
std::vector<String> text;
for (const auto & onexpr : clauses)
text.push_back(onexpr.formatDebug());
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have only one join clause, got {}: [{}], query: '{}'",
clauses.size(), fmt::join(text, " | "), queryToString(table_join));
}
}
} }

View File

@ -11,10 +11,16 @@
#include <DataStreams/SizeLimits.h> #include <DataStreams/SizeLimits.h>
#include <DataTypes/getLeastSupertype.h> #include <DataTypes/getLeastSupertype.h>
#include <Storages/IStorage_fwd.h> #include <Storages/IStorage_fwd.h>
#include <Common/Exception.h>
#include <Parsers/IAST_fwd.h>
#include <cstddef>
#include <unordered_map>
#include <utility> #include <utility>
#include <memory> #include <memory>
#include <common/types.h>
#include <common/logger_useful.h>
namespace DB namespace DB
{ {
@ -47,6 +53,43 @@ class TableJoin
public: public:
using NameToTypeMap = std::unordered_map<String, DataTypePtr>; using NameToTypeMap = std::unordered_map<String, DataTypePtr>;
/// Corresponds to one disjunct
struct JoinOnClause
{
Names key_names_left;
Names key_names_right; /// Duplicating right key names are qualified.
ASTPtr on_filter_condition_left;
ASTPtr on_filter_condition_right;
JoinOnClause() = default;
std::pair<String, String> condColumnNames() const
{
std::pair<String, String> res;
if (on_filter_condition_left)
res.first = on_filter_condition_left->getColumnName();
if (on_filter_condition_right)
res.second = on_filter_condition_right->getColumnName();
return res;
}
size_t keysCount() const
{
assert(key_names_left.size() == key_names_right.size());
return key_names_right.size();
}
String formatDebug() const
{
return fmt::format("Left keys: [{}] Right keys [{}] Condition columns: '{}', '{}'",
fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "),
condColumnNames().first, condColumnNames().second);
}
};
using Clauses = std::vector<JoinOnClause>;
private: private:
/** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
* The join is made by column k. * The join is made by column k.
@ -73,14 +116,14 @@ private:
const size_t max_files_to_merge = 0; const size_t max_files_to_merge = 0;
const String temporary_files_codec = "LZ4"; const String temporary_files_codec = "LZ4";
Names key_names_left; /// the limit has no technical reasons, it supposed to improve safety
Names key_names_right; /// Duplicating names are qualified. const size_t MAX_DISJUNCTS = 16;
ASTs on_filter_condition_asts_left;
ASTs on_filter_condition_asts_right;
ASTs key_asts_left; ASTs key_asts_left;
ASTs key_asts_right; ASTs key_asts_right;
Clauses clauses;
ASTTableJoin table_join; ASTTableJoin table_join;
ASOF::Inequality asof_inequality = ASOF::Inequality::GreaterOrEquals; ASOF::Inequality asof_inequality = ASOF::Inequality::GreaterOrEquals;
@ -112,7 +155,11 @@ private:
/// Create converting actions and change key column names if required /// Create converting actions and change key column names if required
ActionsDAGPtr applyKeyConvertToTable( ActionsDAGPtr applyKeyConvertToTable(
const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, Names & names_to_rename) const; const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NameToNameMap & key_column_rename) const;
void addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr);
void assertHasOneOnExpr() const;
/// Calculates common supertypes for corresponding join key columns. /// Calculates common supertypes for corresponding join key columns.
template <typename LeftNamesAndTypes, typename RightNamesAndTypes> template <typename LeftNamesAndTypes, typename RightNamesAndTypes>
@ -122,17 +169,18 @@ private:
public: public:
TableJoin() = default; TableJoin() = default;
TableJoin(const Settings &, VolumePtr tmp_volume);
TableJoin(const Settings & settings, VolumePtr tmp_volume_);
/// for StorageJoin /// for StorageJoin
TableJoin(SizeLimits limits, bool use_nulls, ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness, TableJoin(SizeLimits limits, bool use_nulls, ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness,
const Names & key_names_right_) const Names & key_names_right)
: size_limits(limits) : size_limits(limits)
, default_max_bytes(0) , default_max_bytes(0)
, join_use_nulls(use_nulls) , join_use_nulls(use_nulls)
, join_algorithm(JoinAlgorithm::HASH) , join_algorithm(JoinAlgorithm::HASH)
, key_names_right(key_names_right_)
{ {
clauses.emplace_back().key_names_right = key_names_right;
table_join.kind = kind; table_join.kind = kind;
table_join.strictness = strictness; table_join.strictness = strictness;
} }
@ -162,8 +210,21 @@ public:
bool enablePartialMergeJoinOptimizations() const { return partial_merge_join_optimizations; } bool enablePartialMergeJoinOptimizations() const { return partial_merge_join_optimizations; }
bool needStreamWithNonJoinedRows() const; bool needStreamWithNonJoinedRows() const;
bool oneDisjunct() const;
JoinOnClause & getOnlyClause() { assertHasOneOnExpr(); return clauses[0]; }
const JoinOnClause & getOnlyClause() const { assertHasOneOnExpr(); return clauses[0]; }
std::vector<JoinOnClause> & getClauses() { return clauses; }
const std::vector<JoinOnClause> & getClauses() const { return clauses; }
Names getAllNames(JoinTableSide side) const;
void resetCollected(); void resetCollected();
void addUsingKey(const ASTPtr & ast); void addUsingKey(const ASTPtr & ast);
void addDisjunct();
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
/* Conditions for left/right table from JOIN ON section. /* Conditions for left/right table from JOIN ON section.
@ -180,8 +241,6 @@ public:
* doesn't supported yet, it can be added later. * doesn't supported yet, it can be added later.
*/ */
void addJoinCondition(const ASTPtr & ast, bool is_left); void addJoinCondition(const ASTPtr & ast, bool is_left);
ASTPtr joinConditionColumn(JoinTableSide side) const;
std::pair<String, String> joinConditionColumnNames() const;
bool hasUsing() const { return table_join.using_expression_list != nullptr; } bool hasUsing() const { return table_join.using_expression_list != nullptr; }
bool hasOn() const { return table_join.on_expression != nullptr; } bool hasOn() const { return table_join.on_expression != nullptr; }
@ -211,8 +270,6 @@ public:
ASTPtr leftKeysList() const; ASTPtr leftKeysList() const;
ASTPtr rightKeysList() const; /// For ON syntax only ASTPtr rightKeysList() const; /// For ON syntax only
const Names & keyNamesLeft() const { return key_names_left; }
const Names & keyNamesRight() const { return key_names_right; }
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
Names columnsAddedByJoin() const Names columnsAddedByJoin() const
@ -224,7 +281,7 @@ public:
} }
/// StorageJoin overrides key names (cause of different names qualification) /// StorageJoin overrides key names (cause of different names qualification)
void setRightKeys(const Names & keys) { key_names_right = keys; } void setRightKeys(const Names & keys) { getOnlyClause().key_names_right = keys; }
Block getRequiredRightKeys(const Block & right_table_keys, std::vector<String> & keys_sources) const; Block getRequiredRightKeys(const Block & right_table_keys, std::vector<String> & keys_sources) const;

View File

@ -1,6 +1,8 @@
#include <Core/Settings.h> #include <Core/Settings.h>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <Common/checkStackSize.h>
#include <Interpreters/TreeRewriter.h> #include <Interpreters/TreeRewriter.h>
#include <Interpreters/LogicalExpressionsOptimizer.h> #include <Interpreters/LogicalExpressionsOptimizer.h>
#include <Interpreters/QueryAliasesVisitor.h> #include <Interpreters/QueryAliasesVisitor.h>
@ -515,6 +517,8 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_
if (table_join.using_expression_list) if (table_join.using_expression_list)
{ {
const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>(); const auto & keys = table_join.using_expression_list->as<ASTExpressionList &>();
analyzed_join.addDisjunct();
for (const auto & key : keys.children) for (const auto & key : keys.children)
analyzed_join.addUsingKey(key); analyzed_join.addUsingKey(key);
} }
@ -523,15 +527,45 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_
bool is_asof = (table_join.strictness == ASTTableJoin::Strictness::Asof); bool is_asof = (table_join.strictness == ASTTableJoin::Strictness::Asof);
CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof}; CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof};
CollectJoinOnKeysVisitor(data).visit(table_join.on_expression); if (auto * or_func = table_join.on_expression->as<ASTFunction>(); or_func && or_func->name == "or")
if (analyzed_join.keyNamesLeft().empty())
{ {
throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression), for (auto & disjunct : or_func->arguments->children)
ErrorCodes::INVALID_JOIN_ON_EXPRESSION); {
analyzed_join.addDisjunct();
CollectJoinOnKeysVisitor(data).visit(disjunct);
}
assert(analyzed_join.getClauses().size() == or_func->arguments->children.size());
}
else
{
analyzed_join.addDisjunct();
CollectJoinOnKeysVisitor(data).visit(table_join.on_expression);
assert(analyzed_join.oneDisjunct());
}
if (analyzed_join.getClauses().empty())
throw DB::Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"Cannot get JOIN keys from JOIN ON section: '{}'",
queryToString(table_join.on_expression));
for (const auto & onexpr : analyzed_join.getClauses())
{
if (onexpr.key_names_left.empty())
throw DB::Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
"Cannot get JOIN keys from JOIN ON section: '{}'",
queryToString(table_join.on_expression));
} }
if (is_asof) if (is_asof)
{
if (!analyzed_join.oneDisjunct())
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "ASOF join doesn't support multiple ORs for keys in JOIN ON section");
data.asofToJoinKeys(); data.asofToJoinKeys();
}
if (!analyzed_join.oneDisjunct() && !analyzed_join.forceHashJoin())
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section");
} }
} }
@ -864,6 +898,10 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
} }
required_source_columns.swap(source_columns); required_source_columns.swap(source_columns);
for (const auto & column : required_source_columns)
{
source_column_names.insert(column.name);
}
} }
NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const

View File

@ -100,4 +100,35 @@ inline bool joinDispatch(ASTTableJoin::Kind kind, ASTTableJoin::Strictness stric
}); });
} }
/// Call function on specified join map
template <typename MapsVariant, typename Func>
inline bool joinDispatch(ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness, std::vector<const MapsVariant *> & mapsv, Func && func)
{
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
{
// NOTE: Avoid using nested static loop as GCC and CLANG have bugs in different ways
// See https://stackoverflow.com/questions/44386415/gcc-and-clang-disagree-about-c17-constexpr-lambda-captures
constexpr auto i = ij / STRICTNESSES.size();
constexpr auto j = ij % STRICTNESSES.size();
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
{
using MapType = typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map;
std::vector<const MapType *> v;
for (const auto & el : mapsv)
{
v.push_back(&std::get<MapType>(*el));
}
func(
std::integral_constant<ASTTableJoin::Kind, KINDS[i]>(),
std::integral_constant<ASTTableJoin::Strictness, STRICTNESSES[j]>(),
v
/*std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps)*/);
return true;
}
return false;
});
}
} }

View File

@ -15,7 +15,6 @@
#include <Interpreters/ActionsDAG.h> #include <Interpreters/ActionsDAG.h>
#include <Interpreters/TableJoin.h> #include <Interpreters/TableJoin.h>
#include <common/logger_useful.h>
namespace DB namespace DB
{ {
@ -194,6 +193,16 @@ void convertColumnsToNullable(Block & block, size_t starting_pos)
convertColumnToNullable(block.getByPosition(i)); convertColumnToNullable(block.getByPosition(i));
} }
void convertColumnsToNullable(MutableColumns & mutable_columns, size_t starting_pos)
{
for (size_t i = starting_pos; i < mutable_columns.size(); ++i)
{
ColumnPtr column = std::move(mutable_columns[i]);
column = makeNullable(column);
mutable_columns[i] = IColumn::mutate(std::move(column));
}
}
/// @warning It assumes that every NULL has default value in nested column (or it does not matter) /// @warning It assumes that every NULL has default value in nested column (or it does not matter)
void removeColumnNullability(ColumnWithTypeAndName & column) void removeColumnNullability(ColumnWithTypeAndName & column)
{ {
@ -291,6 +300,21 @@ ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names)
return ptrs; return ptrs;
} }
ColumnRawPtrMap materializeColumnsInplaceMap(Block & block, const Names & names)
{
ColumnRawPtrMap ptrs;
ptrs.reserve(names.size());
for (const auto & column_name : names)
{
auto & column = block.getByName(column_name).column;
column = recursiveRemoveLowCardinality(column->convertToFullColumnIfConst());
ptrs[column_name] = column.get();
}
return ptrs;
}
ColumnPtr materializeColumn(const Block & block, const String & column_name) ColumnPtr materializeColumn(const Block & block, const String & column_name)
{ {
const auto & src_column = block.getByName(column_name).column; const auto & src_column = block.getByName(column_name).column;
@ -528,7 +552,6 @@ NotJoinedBlocks::NotJoinedBlocks(std::unique_ptr<RightColumnsFiller> filler_,
, saved_block_sample(filler->getEmptyBlock()) , saved_block_sample(filler->getEmptyBlock())
, result_sample_block(materializeBlock(result_sample_block_)) , result_sample_block(materializeBlock(result_sample_block_))
{ {
for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos) for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos)
{ {
/// We need right 'x' for 'RIGHT JOIN ... USING(x)' /// We need right 'x' for 'RIGHT JOIN ... USING(x)'

View File

@ -12,7 +12,9 @@ namespace DB
struct ColumnWithTypeAndName; struct ColumnWithTypeAndName;
class TableJoin; class TableJoin;
class IColumn; class IColumn;
using ColumnRawPtrs = std::vector<const IColumn *>; using ColumnRawPtrs = std::vector<const IColumn *>;
using ColumnRawPtrMap = std::unordered_map<String, const IColumn *>;
using UInt8ColumnDataPtr = const ColumnUInt8::Container *; using UInt8ColumnDataPtr = const ColumnUInt8::Container *;
namespace JoinCommon namespace JoinCommon
@ -21,12 +23,14 @@ bool canBecomeNullable(const DataTypePtr & type);
DataTypePtr convertTypeToNullable(const DataTypePtr & type); DataTypePtr convertTypeToNullable(const DataTypePtr & type);
void convertColumnToNullable(ColumnWithTypeAndName & column); void convertColumnToNullable(ColumnWithTypeAndName & column);
void convertColumnsToNullable(Block & block, size_t starting_pos = 0); void convertColumnsToNullable(Block & block, size_t starting_pos = 0);
void convertColumnsToNullable(MutableColumns & mutable_columns, size_t starting_pos = 0);
void removeColumnNullability(ColumnWithTypeAndName & column); void removeColumnNullability(ColumnWithTypeAndName & column);
void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_column); void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_column);
ColumnPtr emptyNotNullableClone(const ColumnPtr & column); ColumnPtr emptyNotNullableClone(const ColumnPtr & column);
ColumnPtr materializeColumn(const Block & block, const String & name); ColumnPtr materializeColumn(const Block & block, const String & name);
Columns materializeColumns(const Block & block, const Names & names); Columns materializeColumns(const Block & block, const Names & names);
ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names); ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names);
ColumnRawPtrMap materializeColumnsInplaceMap(Block & block, const Names & names);
ColumnRawPtrs getRawPointers(const Columns & columns); ColumnRawPtrs getRawPointers(const Columns & columns);
void removeLowCardinalityInplace(Block & block); void removeLowCardinalityInplace(Block & block);
void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type = true); void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type = true);

View File

@ -217,16 +217,19 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
settings.ostr << nl_or_nothing << indent_str << ")"; settings.ostr << nl_or_nothing << indent_str << ")";
return; return;
} }
/// Should this function to be written as operator? /// Should this function to be written as operator?
bool written = false; bool written = false;
if (arguments && !parameters) if (arguments && !parameters)
{ {
/// Unary prefix operators.
if (arguments->children.size() == 1) if (arguments->children.size() == 1)
{ {
const char * operators[] = const char * operators[] =
{ {
"negate", "-", "negate", "-",
"not", "NOT ", "not", "NOT ",
nullptr nullptr
}; };
@ -270,6 +273,32 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
} }
} }
/// Unary postfix operators.
if (!written && arguments->children.size() == 1)
{
const char * operators[] =
{
"isNull", " IS NULL",
"isNotNull", " IS NOT NULL",
nullptr
};
for (const char ** func = operators; *func; func += 2)
{
if (strcasecmp(name.c_str(), func[0]) != 0)
{
continue;
}
arguments->formatImpl(settings, state, nested_need_parens);
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
written = true;
break;
}
}
/** need_parens - do we need parentheses around the expression with the operator. /** need_parens - do we need parentheses around the expression with the operator.
* They are needed only if this expression is included in another expression with the operator. * They are needed only if this expression is included in another expression with the operator.
*/ */

View File

@ -13,7 +13,7 @@ try
std::string input = std::string(reinterpret_cast<const char*>(data), size); std::string input = std::string(reinterpret_cast<const char*>(data), size);
DB::ParserCreateQuery parser; DB::ParserCreateQuery parser;
DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000);
DB::WriteBufferFromOwnString wb; DB::WriteBufferFromOwnString wb;
DB::formatAST(*ast, wb); DB::formatAST(*ast, wb);

View File

@ -12,7 +12,7 @@ try
std::string input = std::string(reinterpret_cast<const char*>(data), size); std::string input = std::string(reinterpret_cast<const char*>(data), size);
DB::ParserQueryWithOutput parser(input.data() + input.size()); DB::ParserQueryWithOutput parser(input.data() + input.size());
DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000);
DB::WriteBufferFromOwnString wb; DB::WriteBufferFromOwnString wb;
DB::formatAST(*ast, wb); DB::formatAST(*ast, wb);

View File

@ -41,6 +41,7 @@ public:
void finish() void finish()
{ {
std::unique_lock lock(mutex);
is_finished = true; is_finished = true;
condvar.notify_all(); condvar.notify_all();
} }
@ -64,7 +65,7 @@ protected:
private: private:
Chunk data; Chunk data;
bool has_data = false; bool has_data = false;
std::atomic_bool is_finished = false; bool is_finished = false;
std::mutex mutex; std::mutex mutex;
std::condition_variable condvar; std::condition_variable condvar;
}; };

View File

@ -197,12 +197,13 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
/// Push down is for left table only. We need to update JoinStep for push down into right. /// Push down is for left table only. We need to update JoinStep for push down into right.
/// Only inner and left join are supported. Other types may generate default values for left table keys. /// Only inner and left join are supported. Other types may generate default values for left table keys.
/// So, if we push down a condition like `key != 0`, not all rows may be filtered. /// So, if we push down a condition like `key != 0`, not all rows may be filtered.
if (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == ASTTableJoin::Kind::Left) if (table_join.oneDisjunct() && (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == ASTTableJoin::Kind::Left))
{ {
const auto & left_header = join->getInputStreams().front().header; const auto & left_header = join->getInputStreams().front().header;
const auto & res_header = join->getOutputStream().header; const auto & res_header = join->getOutputStream().header;
Names allowed_keys; Names allowed_keys;
for (const auto & name : table_join.keyNamesLeft()) const auto & key_names_left = table_join.getOnlyClause().key_names_left;
for (const auto & name : key_names_left)
{ {
/// Skip key if it is renamed. /// Skip key if it is renamed.
/// I don't know if it is possible. Just in case. /// I don't know if it is possible. Just in case.

View File

@ -30,6 +30,7 @@ TotalsHavingStep::TotalsHavingStep(
bool overflow_row_, bool overflow_row_,
const ActionsDAGPtr & actions_dag_, const ActionsDAGPtr & actions_dag_,
const std::string & filter_column_, const std::string & filter_column_,
bool remove_filter_,
TotalsMode totals_mode_, TotalsMode totals_mode_,
double auto_include_threshold_, double auto_include_threshold_,
bool final_) bool final_)
@ -38,11 +39,14 @@ TotalsHavingStep::TotalsHavingStep(
TotalsHavingTransform::transformHeader( TotalsHavingTransform::transformHeader(
input_stream_.header, input_stream_.header,
actions_dag_.get(), actions_dag_.get(),
filter_column_,
remove_filter_,
final_), final_),
getTraits(!filter_column_.empty())) getTraits(!filter_column_.empty()))
, overflow_row(overflow_row_) , overflow_row(overflow_row_)
, actions_dag(actions_dag_) , actions_dag(actions_dag_)
, filter_column_name(filter_column_) , filter_column_name(filter_column_)
, remove_filter(remove_filter_)
, totals_mode(totals_mode_) , totals_mode(totals_mode_)
, auto_include_threshold(auto_include_threshold_) , auto_include_threshold(auto_include_threshold_)
, final(final_) , final(final_)
@ -58,6 +62,7 @@ void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const
overflow_row, overflow_row,
expression_actions, expression_actions,
filter_column_name, filter_column_name,
remove_filter,
totals_mode, totals_mode,
auto_include_threshold, auto_include_threshold,
final); final);
@ -85,7 +90,10 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr
void TotalsHavingStep::describeActions(FormatSettings & settings) const void TotalsHavingStep::describeActions(FormatSettings & settings) const
{ {
String prefix(settings.offset, ' '); String prefix(settings.offset, ' ');
settings.out << prefix << "Filter column: " << filter_column_name << '\n'; settings.out << prefix << "Filter column: " << filter_column_name;
if (remove_filter)
settings.out << " (removed)";
settings.out << '\n';
settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n'; settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n';
if (actions_dag) if (actions_dag)

View File

@ -18,6 +18,7 @@ public:
bool overflow_row_, bool overflow_row_,
const ActionsDAGPtr & actions_dag_, const ActionsDAGPtr & actions_dag_,
const std::string & filter_column_, const std::string & filter_column_,
bool remove_filter_,
TotalsMode totals_mode_, TotalsMode totals_mode_,
double auto_include_threshold_, double auto_include_threshold_,
bool final_); bool final_);
@ -35,6 +36,7 @@ private:
bool overflow_row; bool overflow_row;
ActionsDAGPtr actions_dag; ActionsDAGPtr actions_dag;
String filter_column_name; String filter_column_name;
bool remove_filter;
TotalsMode totals_mode; TotalsMode totals_mode;
double auto_include_threshold; double auto_include_threshold;
bool final; bool final;

View File

@ -124,7 +124,8 @@ void JoiningTransform::work()
return; return;
} }
non_joined_blocks = join->getNonJoinedBlocks(outputs.front().getHeader(), max_block_size); non_joined_blocks = join->getNonJoinedBlocks(
inputs.front().getHeader(), outputs.front().getHeader(), max_block_size);
if (!non_joined_blocks) if (!non_joined_blocks)
{ {
process_non_joined = false; process_non_joined = false;

View File

@ -28,13 +28,22 @@ void finalizeChunk(Chunk & chunk)
chunk.setColumns(std::move(columns), num_rows); chunk.setColumns(std::move(columns), num_rows);
} }
Block TotalsHavingTransform::transformHeader(Block block, const ActionsDAG * expression, bool final) Block TotalsHavingTransform::transformHeader(
Block block,
const ActionsDAG * expression,
const std::string & filter_column_name,
bool remove_filter,
bool final)
{ {
if (final) if (final)
finalizeBlock(block); finalizeBlock(block);
if (expression) if (expression)
{
block = expression->updateHeader(std::move(block)); block = expression->updateHeader(std::move(block));
if (remove_filter)
block.erase(filter_column_name);
}
return block; return block;
} }
@ -44,20 +53,19 @@ TotalsHavingTransform::TotalsHavingTransform(
bool overflow_row_, bool overflow_row_,
const ExpressionActionsPtr & expression_, const ExpressionActionsPtr & expression_,
const std::string & filter_column_, const std::string & filter_column_,
bool remove_filter_,
TotalsMode totals_mode_, TotalsMode totals_mode_,
double auto_include_threshold_, double auto_include_threshold_,
bool final_) bool final_)
: ISimpleTransform(header, transformHeader(header, expression_ ? &expression_->getActionsDAG() : nullptr, final_), true) : ISimpleTransform(header, transformHeader(header, expression_ ? &expression_->getActionsDAG() : nullptr, filter_column_, remove_filter_, final_), true)
, overflow_row(overflow_row_) , overflow_row(overflow_row_)
, expression(expression_) , expression(expression_)
, filter_column_name(filter_column_) , filter_column_name(filter_column_)
, remove_filter(remove_filter_)
, totals_mode(totals_mode_) , totals_mode(totals_mode_)
, auto_include_threshold(auto_include_threshold_) , auto_include_threshold(auto_include_threshold_)
, final(final_) , final(final_)
{ {
if (!filter_column_name.empty())
filter_column_pos = outputs.front().getHeader().getPositionByName(filter_column_name);
finalized_header = getInputPort().getHeader(); finalized_header = getInputPort().getHeader();
finalizeBlock(finalized_header); finalizeBlock(finalized_header);
@ -67,10 +75,17 @@ TotalsHavingTransform::TotalsHavingTransform(
auto totals_header = finalized_header; auto totals_header = finalized_header;
size_t num_rows = totals_header.rows(); size_t num_rows = totals_header.rows();
expression->execute(totals_header, num_rows); expression->execute(totals_header, num_rows);
filter_column_pos = totals_header.getPositionByName(filter_column_name);
if (remove_filter)
totals_header.erase(filter_column_name);
outputs.emplace_back(totals_header, this); outputs.emplace_back(totals_header, this);
} }
else else
{
if (!filter_column_name.empty())
filter_column_pos = finalized_header.getPositionByName(filter_column_name);
outputs.emplace_back(finalized_header, this); outputs.emplace_back(finalized_header, this);
}
/// Initialize current totals with initial state. /// Initialize current totals with initial state.
current_totals.reserve(header.columns()); current_totals.reserve(header.columns());
@ -167,9 +182,11 @@ void TotalsHavingTransform::transform(Chunk & chunk)
} }
expression->execute(finalized_block, num_rows); expression->execute(finalized_block, num_rows);
ColumnPtr filter_column_ptr = finalized_block.getByPosition(filter_column_pos).column;
if (remove_filter)
finalized_block.erase(filter_column_name);
auto columns = finalized_block.getColumns(); auto columns = finalized_block.getColumns();
ColumnPtr filter_column_ptr = columns[filter_column_pos];
ConstantFilterDescription const_filter_description(*filter_column_ptr); ConstantFilterDescription const_filter_description(*filter_column_ptr);
if (const_filter_description.always_true) if (const_filter_description.always_true)
@ -270,6 +287,8 @@ void TotalsHavingTransform::prepareTotals()
size_t num_rows = totals.getNumRows(); size_t num_rows = totals.getNumRows();
auto block = finalized_header.cloneWithColumns(totals.detachColumns()); auto block = finalized_header.cloneWithColumns(totals.detachColumns());
expression->execute(block, num_rows); expression->execute(block, num_rows);
if (remove_filter)
block.erase(filter_column_name);
/// Note: after expression totals may have several rows if `arrayJoin` was used in expression. /// Note: after expression totals may have several rows if `arrayJoin` was used in expression.
totals = Chunk(block.getColumns(), num_rows); totals = Chunk(block.getColumns(), num_rows);
} }

View File

@ -28,6 +28,7 @@ public:
bool overflow_row_, bool overflow_row_,
const ExpressionActionsPtr & expression_, const ExpressionActionsPtr & expression_,
const std::string & filter_column_, const std::string & filter_column_,
bool remove_filter_,
TotalsMode totals_mode_, TotalsMode totals_mode_,
double auto_include_threshold_, double auto_include_threshold_,
bool final_); bool final_);
@ -39,7 +40,7 @@ public:
Status prepare() override; Status prepare() override;
void work() override; void work() override;
static Block transformHeader(Block block, const ActionsDAG * expression, bool final); static Block transformHeader(Block block, const ActionsDAG * expression, const std::string & filter_column_name, bool remove_filter, bool final);
protected: protected:
void transform(Chunk & chunk) override; void transform(Chunk & chunk) override;
@ -55,6 +56,7 @@ private:
bool overflow_row; bool overflow_row;
ExpressionActionsPtr expression; ExpressionActionsPtr expression;
String filter_column_name; String filter_column_name;
bool remove_filter;
TotalsMode totals_mode; TotalsMode totals_mode;
double auto_include_threshold; double auto_include_threshold;
bool final; bool final;

View File

@ -408,7 +408,6 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn(
auto element_key = literal->value; auto element_key = literal->value;
const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type);
out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key))); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key)));
return true;
} }
else else
{ {

View File

@ -212,7 +212,7 @@ MergeTreeConditionFullText::MergeTreeConditionFullText(
query_info, context, query_info, context,
[this] (const ASTPtr & node, ContextPtr /* context */, Block & block_with_constants, RPNElement & out) -> bool [this] (const ASTPtr & node, ContextPtr /* context */, Block & block_with_constants, RPNElement & out) -> bool
{ {
return this->atomFromAST(node, block_with_constants, out); return this->traverseAtomAST(node, block_with_constants, out);
}).extractRPN()); }).extractRPN());
} }
@ -355,9 +355,9 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx
return rpn_stack[0].can_be_true; return rpn_stack[0].can_be_true;
} }
bool MergeTreeConditionFullText::getKey(const ASTPtr & node, size_t & key_column_num) bool MergeTreeConditionFullText::getKey(const std::string & key_column_name, size_t & key_column_num)
{ {
auto it = std::find(index_columns.begin(), index_columns.end(), node->getColumnName()); auto it = std::find(index_columns.begin(), index_columns.end(), key_column_name);
if (it == index_columns.end()) if (it == index_columns.end())
return false; return false;
@ -365,153 +365,243 @@ bool MergeTreeConditionFullText::getKey(const ASTPtr & node, size_t & key_column
return true; return true;
} }
bool MergeTreeConditionFullText::atomFromAST( bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out)
const ASTPtr & node, Block & block_with_constants, RPNElement & out)
{ {
Field const_value;
DataTypePtr const_type;
if (const auto * func = typeid_cast<const ASTFunction *>(node.get()))
{ {
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children; Field const_value;
DataTypePtr const_type;
if (args.size() != 2) if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type))
return false;
size_t key_arg_pos; /// Position of argument with key column (non-const argument)
size_t key_column_num = -1; /// Number of a key column (inside key_column_names array)
const auto & func_name = func->name;
if (functionIsInOrGlobalInOperator(func_name) && tryPrepareSetBloomFilter(args, out))
{ {
key_arg_pos = 0; /// Check constant like in KeyCondition
} if (const_value.getType() == Field::Types::UInt64
else if (KeyCondition::getConstant(args[1], block_with_constants, const_value, const_type) && getKey(args[0], key_column_num)) || const_value.getType() == Field::Types::Int64
{ || const_value.getType() == Field::Types::Float64)
key_arg_pos = 0;
}
else if (KeyCondition::getConstant(args[0], block_with_constants, const_value, const_type) && getKey(args[1], key_column_num))
{
key_arg_pos = 1;
}
else
return false;
if (const_type && const_type->getTypeId() != TypeIndex::String
&& const_type->getTypeId() != TypeIndex::FixedString
&& const_type->getTypeId() != TypeIndex::Array)
{
return false;
}
if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals"))
return false;
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;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
stringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "equals")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (func_name == "like")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
likeStringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "notLike")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
likeStringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "hasToken")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
stringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (func_name == "startsWith")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (func_name == "endsWith")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (func_name == "multiSearchAny")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_MULTI_SEARCH;
/// 2d vector is not needed here but is used because already exists for FUNCTION_IN
std::vector<std::vector<BloomFilter>> bloom_filters;
bloom_filters.emplace_back();
for (const auto & element : const_value.get<Array>())
{ {
if (element.getType() != Field::Types::String) /// Zero in all types is represented in memory the same way as in UInt64.
return false; out.function = const_value.get<UInt64>()
? RPNElement::ALWAYS_TRUE
: RPNElement::ALWAYS_FALSE;
bloom_filters.back().emplace_back(params); return true;
stringToBloomFilter(element.get<String>(), token_extractor, bloom_filters.back().back());
} }
out.set_bloom_filters = std::move(bloom_filters);
return true;
} }
else if (func_name == "notIn")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_IN;
return true;
}
else if (func_name == "in")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_IN;
return true;
}
return false;
} }
else if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type))
{
/// Check constant like in KeyCondition
if (const_value.getType() == Field::Types::UInt64
|| const_value.getType() == Field::Types::Int64
|| const_value.getType() == Field::Types::Float64)
{
/// Zero in all types is represented in memory the same way as in UInt64.
out.function = const_value.get<UInt64>()
? RPNElement::ALWAYS_TRUE
: RPNElement::ALWAYS_FALSE;
return true; if (const auto * function = node->as<ASTFunction>())
{
if (!function->arguments)
return false;
const ASTs & arguments = function->arguments->children;
if (arguments.size() != 2)
return false;
if (functionIsInOrGlobalInOperator(function->name))
{
if (tryPrepareSetBloomFilter(arguments, out))
{
if (function->name == "notIn")
{
out.function = RPNElement::FUNCTION_NOT_IN;
return true;
}
else if (function->name == "in")
{
out.function = RPNElement::FUNCTION_IN;
return true;
}
}
} }
else if (function->name == "equals" ||
function->name == "notEquals" ||
function->name == "has" ||
function->name == "mapContains" ||
function->name == "like" ||
function->name == "notLike" ||
function->name == "hasToken" ||
function->name == "startsWith" ||
function->name == "endsWith" ||
function->name == "multiSearchAny")
{
Field const_value;
DataTypePtr const_type;
if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type))
{
if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out))
return true;
}
else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type) && (function->name == "equals" || function->name == "notEquals"))
{
if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out))
return true;
}
}
}
return false;
}
bool MergeTreeConditionFullText::traverseASTEquals(
const String & function_name,
const ASTPtr & key_ast,
const DataTypePtr & value_type,
const Field & value_field,
RPNElement & out)
{
auto value_data_type = WhichDataType(value_type);
if (!value_data_type.isStringOrFixedString() && !value_data_type.isArray())
return false;
if (!token_extractor->supportLike() && (function_name == "like" || function_name == "notLike"))
return false;
Field const_value = value_field;
size_t key_column_num = 0;
bool key_exists = getKey(key_ast->getColumnName(), key_column_num);
bool map_key_exists = getKey(fmt::format("mapKeys({})", key_ast->getColumnName()), key_column_num);
if (const auto * function = key_ast->as<ASTFunction>())
{
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 & map_column_name = assert_cast<ASTIdentifier *>(function->arguments.get()->children[0].get())->name();
size_t map_keys_key_column_num = 0;
auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name);
bool map_keys_exists = getKey(map_keys_index_column_name, map_keys_key_column_num);
size_t map_values_key_column_num = 0;
auto map_values_index_column_name = fmt::format("mapValues({})", map_column_name);
bool map_values_exists = getKey(map_values_index_column_name, map_values_key_column_num);
if (map_keys_exists)
{
auto & argument = function->arguments.get()->children[1];
if (const auto * literal = argument->as<ASTLiteral>())
{
auto element_key = literal->value;
const_value = element_key;
key_column_num = map_keys_key_column_num;
key_exists = true;
}
else
{
return false;
}
}
else if (map_values_exists)
{
key_column_num = map_values_key_column_num;
key_exists = true;
}
else
{
return false;
}
}
}
if (!key_exists && !map_key_exists)
return false;
if (map_key_exists && (function_name == "has" || function_name == "mapContains"))
{
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;
}
else if (function_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 (function_name == "notEquals")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
stringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (function_name == "equals")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (function_name == "like")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
likeStringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (function_name == "notLike")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_NOT_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
likeStringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (function_name == "hasToken")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_EQUALS;
out.bloom_filter = std::make_unique<BloomFilter>(params);
stringToBloomFilter(const_value.get<String>(), token_extractor, *out.bloom_filter);
return true;
}
else if (function_name == "startsWith")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (function_name == "endsWith")
{
out.key_column = key_column_num;
return createFunctionEqualsCondition(out, const_value, params, token_extractor);
}
else if (function_name == "multiSearchAny")
{
out.key_column = key_column_num;
out.function = RPNElement::FUNCTION_MULTI_SEARCH;
/// 2d vector is not needed here but is used because already exists for FUNCTION_IN
std::vector<std::vector<BloomFilter>> bloom_filters;
bloom_filters.emplace_back();
for (const auto & element : const_value.get<Array>())
{
if (element.getType() != Field::Types::String)
return false;
bloom_filters.back().emplace_back(params);
stringToBloomFilter(element.get<String>(), token_extractor, bloom_filters.back().back());
}
out.set_bloom_filters = std::move(bloom_filters);
return true;
} }
return false; return false;
@ -534,7 +624,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
for (size_t i = 0; i < tuple_elements.size(); ++i) for (size_t i = 0; i < tuple_elements.size(); ++i)
{ {
size_t key = 0; size_t key = 0;
if (getKey(tuple_elements[i], key)) if (getKey(tuple_elements[i]->getColumnName(), key))
{ {
key_tuple_mapping.emplace_back(i, key); key_tuple_mapping.emplace_back(i, key);
data_types.push_back(index_data_types[key]); data_types.push_back(index_data_types[key]);
@ -544,7 +634,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter(
else else
{ {
size_t key = 0; size_t key = 0;
if (getKey(left_arg, key)) if (getKey(left_arg->getColumnName(), key))
{ {
key_tuple_mapping.emplace_back(0, key); key_tuple_mapping.emplace_back(0, key);
data_types.push_back(index_data_types[key]); data_types.push_back(index_data_types[key]);

View File

@ -146,9 +146,16 @@ private:
using RPN = std::vector<RPNElement>; using RPN = std::vector<RPNElement>;
bool atomFromAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out);
bool getKey(const ASTPtr & node, size_t & key_column_num); bool traverseASTEquals(
const String & function_name,
const ASTPtr & key_ast,
const DataTypePtr & value_type,
const Field & value_field,
RPNElement & out);
bool getKey(const std::string & key_column_name, size_t & key_column_num);
bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out); bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out);
static bool createFunctionEqualsCondition( static bool createFunctionEqualsCondition(

View File

@ -377,6 +377,9 @@ public:
, max_block_size(max_block_size_) , max_block_size(max_block_size_)
, sample_block(std::move(sample_block_)) , sample_block(std::move(sample_block_))
{ {
if (!join->getTableJoin().oneDisjunct())
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin does not support OR for keys in JOIN ON section");
column_indices.resize(sample_block.columns()); column_indices.resize(sample_block.columns());
auto & saved_block = join->getJoinedData()->sample_block; auto & saved_block = join->getJoinedData()->sample_block;
@ -410,7 +413,7 @@ protected:
return {}; return {};
Chunk chunk; Chunk chunk;
if (!joinDispatch(join->kind, join->strictness, join->data->maps, if (!joinDispatch(join->kind, join->strictness, join->data->maps.front(),
[&](auto kind, auto strictness, auto & map) { chunk = createChunk<kind, strictness>(map); })) [&](auto kind, auto strictness, auto & map) { chunk = createChunk<kind, strictness>(map); }))
throw Exception("Logical error: unknown JOIN strictness", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: unknown JOIN strictness", ErrorCodes::LOGICAL_ERROR);
return chunk; return chunk;

View File

@ -65,11 +65,20 @@ def signal_handler(sig, frame):
def stop_tests(): def stop_tests():
global stop_tests_triggered_lock global stop_tests_triggered_lock
global stop_tests_triggered global stop_tests_triggered
global restarted_tests
with stop_tests_triggered_lock: with stop_tests_triggered_lock:
if not stop_tests_triggered.is_set(): if not stop_tests_triggered.is_set():
stop_tests_triggered.set() stop_tests_triggered.set()
# materialize multiprocessing.Manager().list() object before
# sending SIGTERM since this object is a proxy, that requires
# communicating with manager thread, but after SIGTERM will be
# send, this thread will die, and you will get
# ConnectionRefusedError error for any access to "restarted_tests"
# variable.
restarted_tests = [*restarted_tests]
# send signal to all processes in group to avoid hung check triggering # send signal to all processes in group to avoid hung check triggering
# (to avoid terminating clickhouse-test itself, the signal should be ignored) # (to avoid terminating clickhouse-test itself, the signal should be ignored)
signal.signal(signal.SIGTERM, signal.SIG_IGN) signal.signal(signal.SIGTERM, signal.SIG_IGN)
@ -465,7 +474,7 @@ class TestCase:
description += "\n{}\n".format(diff) description += "\n{}\n".format(diff)
return TestResult(self.name, TestStatus.FAIL, FailureReason.RESULT_DIFF, total_time, description) return TestResult(self.name, TestStatus.FAIL, FailureReason.RESULT_DIFF, total_time, description)
if self.testcase_args.test_runs > 1 and total_time > 60 and 'long' not in self.name: if self.testcase_args.test_runs > 1 and total_time > 60 and 'long' not in self.tags:
# We're in Flaky Check mode, check the run time as well while we're at it. # We're in Flaky Check mode, check the run time as well while we're at it.
return TestResult(self.name, TestStatus.FAIL, FailureReason.TOO_LONG, total_time, description) return TestResult(self.name, TestStatus.FAIL, FailureReason.TOO_LONG, total_time, description)

View File

@ -46,6 +46,7 @@ ln -sf $SRC_PATH/users.d/database_atomic_drop_detach_sync.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/
# FIXME DataPartsExchange may hang for http_send_timeout seconds # FIXME DataPartsExchange may hang for http_send_timeout seconds
# when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"),

View File

@ -0,0 +1,8 @@
<yandex>
<profiles>
<default>
<max_untracked_memory>1Mi</max_untracked_memory>
<memory_profiler_step>1Mi</memory_profiler_step>
</default>
</profiles>
</yandex>

View File

@ -8,6 +8,9 @@
<insert_quorum_timeout>60000</insert_quorum_timeout> <insert_quorum_timeout>60000</insert_quorum_timeout>
<!-- 30 seconds (default is 2 minutes) --> <!-- 30 seconds (default is 2 minutes) -->
<replication_wait_for_inactive_replica_timeout>30</replication_wait_for_inactive_replica_timeout> <replication_wait_for_inactive_replica_timeout>30</replication_wait_for_inactive_replica_timeout>
<!-- Increase timeout for async INSERT to avoid flakiness in tests -->
<async_insert_busy_timeout_ms>5000</async_insert_busy_timeout_ms>
</default> </default>
</profiles> </profiles>
</yandex> </yandex>

View File

@ -16,4 +16,6 @@ SELECT k, js1.s, t2.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s F
SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k;
SELECT k, t2.k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; SELECT k, t2.k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k;
SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k OR js1.s == t2.k; -- { serverError 48 }
DROP TABLE t2; DROP TABLE t2;

View File

@ -146,7 +146,7 @@ SELECT
t2_00826.b t2_00826.b
FROM t1_00826 FROM t1_00826
ALL INNER JOIN t2_00826 ON b = t2_00826.a ALL INNER JOIN t2_00826 ON b = t2_00826.a
WHERE (b = t2_00826.a) AND (isNull(t2_00826.b) OR (t2_00826.b > t2_00826.a)) WHERE (b = t2_00826.a) AND (t2_00826.b IS NULL OR (t2_00826.b > t2_00826.a))
--- do not rewrite alias --- --- do not rewrite alias ---
SELECT a AS b SELECT a AS b
FROM t1_00826 FROM t1_00826
@ -178,4 +178,4 @@ SELECT
t2_00826.b t2_00826.b
FROM t1_00826 FROM t1_00826
ALL INNER JOIN t2_00826 ON a = t2_00826.a ALL INNER JOIN t2_00826 ON a = t2_00826.a
WHERE (a = t2_00826.a) AND (isNull(t2_00826.b) OR (t2_00826.b < 2)) WHERE (a = t2_00826.a) AND (t2_00826.b IS NULL OR (t2_00826.b < 2))

View File

@ -3,4 +3,5 @@
0 0
0 0 0 0
0 0 0 0 0 0
0 0 0 0
0 0 0 0 0 0

View File

@ -3,6 +3,7 @@ select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy join syste
select t1.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t1.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
select t2.*, t3.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t2.*, t3.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
select t1.dummy, t2.*, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t1.dummy, t2.*, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy;
select t1.dummy, t2.*, t3.dummy from system.one t1 join (select dummy,0 as another_one from system.one) t2 on t1.dummy = t2.dummy OR t1.dummy = t2.another_one join system.one t3 ON t1.dummy = t3.dummy;
select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1
join system.one t2 on t1.dummy = t2.dummy join system.one t2 on t1.dummy = t2.dummy

View File

@ -4,6 +4,8 @@
4 0 4 0
5 0 5 0
6 0 6 0
7 1 0
8 1 0
- -
1 0 1 0
2 2 a2 2 2 a2

View File

@ -3,6 +3,7 @@ SET joined_subquery_requires_alias = 0;
SELECT ax, c FROM (SELECT [1,2] ax, 0 c) ARRAY JOIN ax JOIN (SELECT 0 c) USING (c); SELECT ax, c FROM (SELECT [1,2] ax, 0 c) ARRAY JOIN ax JOIN (SELECT 0 c) USING (c);
SELECT ax, c FROM (SELECT [3,4] ax, 0 c) JOIN (SELECT 0 c) USING (c) ARRAY JOIN ax; SELECT ax, c FROM (SELECT [3,4] ax, 0 c) JOIN (SELECT 0 c) USING (c) ARRAY JOIN ax;
SELECT ax, c FROM (SELECT [5,6] ax, 0 c) s1 JOIN system.one s2 ON s1.c = s2.dummy ARRAY JOIN ax; SELECT ax, c FROM (SELECT [5,6] ax, 0 c) s1 JOIN system.one s2 ON s1.c = s2.dummy ARRAY JOIN ax;
SELECT ax, c, d FROM (SELECT [7,8] ax, 1 c, 0 d) s1 JOIN system.one s2 ON s1.c = s2.dummy OR s1.d = s2.dummy ARRAY JOIN ax;
SELECT ax, c FROM (SELECT [101,102] ax, 0 c) s1 SELECT ax, c FROM (SELECT [101,102] ax, 0 c) s1
@ -48,4 +49,3 @@ JOIN ( SELECT toInt32(dummy) AS dummy FROM system.one ) AS y USING dummy GROUP B
DROP TABLE IF EXISTS f; DROP TABLE IF EXISTS f;
DROP TABLE IF EXISTS d; DROP TABLE IF EXISTS d;

View File

@ -1,12 +1,13 @@
DROP TABLE IF EXISTS t; DROP TABLE IF EXISTS t;
DROP TABLE IF EXISTS nt; DROP TABLE IF EXISTS nt;
CREATE TABLE t (x String) ENGINE = Memory; CREATE TABLE t (x String) ENGINE = Log();
CREATE TABLE nt (x Nullable(String)) ENGINE = Memory; CREATE TABLE nt (x Nullable(String)) ENGINE = Log();
INSERT INTO t (x) VALUES ('id'), ('1'); INSERT INTO t (x) VALUES ('id'), ('1');
INSERT INTO nt (x) VALUES ('id'), (NULL), ('1'); INSERT INTO nt (x) VALUES ('id'), (NULL), ('1');
SET join_use_nulls = 1; SET join_use_nulls = 1;
SELECT 'on'; SELECT 'on';

View File

@ -0,0 +1,27 @@
on with or
n rj n 1 1
n rj n id id
n rj n \N \N
n a rj n 1 1
n a rj n id id
n a rj n \N \N
n fj n 1 1
n fj n id id
n fj n \N \N
n fj n \N \N
t rj n 1 1
t rj n id id
t rj n \N \N
t fj n 1 1
t fj n id id
t fj n \N \N
n rj t 1 1
n rj t id id
n a rj t 1 1
n a rj t id id
n fj t 1 1
n fj t id id
n fj t \N \N
n fj t 1 1
n fj t id id
n fj t \N \N

View File

@ -0,0 +1,31 @@
DROP TABLE IF EXISTS t;
DROP TABLE IF EXISTS nt;
DROP TABLE IF EXISTS ntxy;
CREATE TABLE t (x String) ENGINE = Log();
CREATE TABLE nt (x Nullable(String)) ENGINE = Log();
CREATE TABLE ntxy (x Nullable(String), y Nullable(String)) ENGINE = Log();
INSERT INTO t (x) VALUES ('id'), ('1');
INSERT INTO nt (x) VALUES ('id'), (NULL), ('1');
INSERT INTO ntxy (x, y) VALUES ('id', 'id'), (NULL, NULL), ('1', '1');
SET join_use_nulls = 1;
SELECT 'on with or';
SELECT 'n rj n', t1.x, t2.x FROM nt AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x;
SELECT 'n a rj n', t1.x, t2.x FROM nt AS t1 ANY RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x;
SELECT 'n fj n', t1.x, t2.x FROM nt AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x;
SELECT 't rj n', t1.x, t2.x FROM t AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x;
SELECT 't fj n', t1.x, t2.x FROM t AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x;
SELECT 'n rj t', t1.x, t2.x FROM ntxy AS t1 RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x;
SELECT 'n a rj t', t1.x, t2.x FROM ntxy AS t1 ANY RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x;
SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t1.x = t2.x OR t2.x = t1.y ORDER BY t1.x;
SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t2.x = t1.y OR t1.x = t2.x ORDER BY t1.x;
DROP TABLE t;
DROP TABLE nt;
DROP TABLE ntxy;

View File

@ -21,5 +21,7 @@ SELECT A.a, A.t, B.b, B.t FROM A ASOF JOIN B ON A.a == B.b AND A.t < B.t ORDER B
SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t == B.t; -- { serverError 403 } SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t == B.t; -- { serverError 403 }
SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t != B.t; -- { serverError 403 } SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t != B.t; -- { serverError 403 }
SELECT A.a, A.t, B.b, B.t FROM A ASOF JOIN B ON A.a == B.b AND A.t < B.t OR A.a == B.b + 1 ORDER BY (A.a, A.t); -- { serverError 48 }
DROP TABLE A; DROP TABLE A;
DROP TABLE B; DROP TABLE B;

View File

@ -1,3 +1,5 @@
1 1 1 1
1 1 1 1
1 1 1 1
1 1
1 1

View File

@ -0,0 +1,39 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
set -e
db="db_$CLICKHOUSE_DATABASE"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS $db;"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE $db;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE $db.\`таблица_со_странным_названием\` (a UInt64, b UInt64) ENGINE = Log;"
$CLICKHOUSE_CLIENT -q "INSERT INTO $db.\`таблица_со_странным_названием\` VALUES (1, 1);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM $db.\`таблица_со_странным_названием\`;"
$CLICKHOUSE_CLIENT -q "DETACH DATABASE $db;"
$CLICKHOUSE_CLIENT -q "ATTACH DATABASE $db;"
$CLICKHOUSE_CLIENT -q "SELECT * FROM $db.\`таблица_со_странным_названием\`;"
$CLICKHOUSE_CLIENT -q "DROP TABLE $db.\`таблица_со_странным_названием\`;"
$CLICKHOUSE_CLIENT -q "DROP DATABASE $db;"
lazy_db="lazy_$CLICKHOUSE_DATABASE"
$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS $lazy_db;"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE $lazy_db ENGINE = Lazy(1);"
$CLICKHOUSE_CLIENT -q "CREATE TABLE $lazy_db.\`таблица_со_странным_названием\` (a UInt64, b UInt64) ENGINE = Log;"
$CLICKHOUSE_CLIENT -q "INSERT INTO $lazy_db.\`таблица_со_странным_названием\` VALUES (1, 1);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM $lazy_db.\`таблица_со_странным_названием\`;"
$CLICKHOUSE_CLIENT -q "DETACH DATABASE $lazy_db;"
$CLICKHOUSE_CLIENT -q "ATTACH DATABASE $lazy_db;"
$CLICKHOUSE_CLIENT -q "SELECT * FROM $lazy_db.\`таблица_со_странным_названием\`;"
$CLICKHOUSE_CLIENT -q "DROP TABLE $lazy_db.\`таблица_со_странным_названием\`;"
$CLICKHOUSE_CLIENT -q "DROP DATABASE $lazy_db;"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE $lazy_db ENGINE = Lazy(10);"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $lazy_db.test;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS $lazy_db.test (a UInt64, b UInt64) ENGINE = Log;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS $lazy_db.test (a UInt64, b UInt64) ENGINE = Log;"
$CLICKHOUSE_CLIENT -q "INSERT INTO $lazy_db.test VALUES (1, 1);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM $lazy_db.test;"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $lazy_db.test;"
$CLICKHOUSE_CLIENT -q "DROP DATABASE $lazy_db;"

View File

@ -1,24 +0,0 @@
-- Tags: no-parallel
DROP DATABASE IF EXISTS testlazy;
CREATE TABLE `таблица_со_странным_названием` (a UInt64, b UInt64) ENGINE = Log;
INSERT INTO `таблица_со_странным_названием` VALUES (1, 1);
SELECT * FROM `таблица_со_странным_названием`;
DROP TABLE `таблица_со_странным_названием`;
CREATE DATABASE testlazy ENGINE = Lazy(1);
CREATE TABLE testlazy.`таблица_со_странным_названием` (a UInt64, b UInt64) ENGINE = Log;
INSERT INTO testlazy.`таблица_со_странным_названием` VALUES (1, 1);
SELECT * FROM testlazy.`таблица_со_странным_названием`;
DROP TABLE testlazy.`таблица_со_странным_названием`;
DROP DATABASE testlazy;
CREATE DATABASE testlazy ENGINE = Lazy(10);
DROP TABLE IF EXISTS testlazy.test;
CREATE TABLE IF NOT EXISTS testlazy.test (a UInt64, b UInt64) ENGINE = Log;
CREATE TABLE IF NOT EXISTS testlazy.test (a UInt64, b UInt64) ENGINE = Log;
INSERT INTO testlazy.test VALUES (1, 1);
SELECT * FROM testlazy.test;
DROP TABLE IF EXISTS testlazy.test;
DROP DATABASE testlazy;

View File

@ -37,6 +37,11 @@ flat: inner on
1 1 1 1 1 1 1 1 1 1
2 2 2 2 2 2 2 2 2 2
3 3 3 3 3 3 3 3 3 3
flat: inner or
0 0 0 0 0
1000 1 1 1 1
2 2 2 2 2
3000 3 3 3 3
hashed: left on hashed: left on
0 0 0 0 0 0 0 0 0 0
1 1 1 1 1 1 1 1 1 1

View File

@ -43,6 +43,8 @@ SELECT 'flat: inner';
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 JOIN dict_flat d USING(key); SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 JOIN dict_flat d USING(key);
SELECT 'flat: inner on'; SELECT 'flat: inner on';
SELECT * FROM (SELECT number AS k FROM numbers(100)) s1 JOIN dict_flat d ON k = key ORDER BY k; SELECT * FROM (SELECT number AS k FROM numbers(100)) s1 JOIN dict_flat d ON k = key ORDER BY k;
SELECT 'flat: inner or'; -- it's not a join over dictionary, because it doen't suppoert multiple keys, but of falls back to regular join
SELECT * FROM (SELECT if(number % 2 = 0, number, number * 1000) AS k FROM numbers(100)) s1 JOIN dict_flat d ON k = key OR k == 1000 * key ORDER BY key;
SET join_use_nulls = 1; SET join_use_nulls = 1;

View File

@ -3,3 +3,5 @@ connect_timeout Seconds 10
connect_timeout_with_failover_ms Milliseconds 2000 connect_timeout_with_failover_ms Milliseconds 2000
connect_timeout_with_failover_secure_ms Milliseconds 3000 connect_timeout_with_failover_secure_ms Milliseconds 3000
max_memory_usage UInt64 10000000000 max_memory_usage UInt64 10000000000
max_untracked_memory UInt64 1048576
memory_profiler_step UInt64 1048576

View File

@ -4,8 +4,20 @@ SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON (A.a = arrayJoin([1])); -- {
SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON equals(a); -- { serverError 62 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON equals(a); -- { serverError 62 }
SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON less(a); -- { serverError 62 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON less(a); -- { serverError 62 }
SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b OR a = b; -- { serverError 403 }
SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a > b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a > b; -- { serverError 403 }
SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a < b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a < b; -- { serverError 403 }
SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a >= b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a >= b; -- { serverError 403 }
SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a <= b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a <= b; -- { serverError 403 }
SET join_algorithm = 'partial_merge';
SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 }
-- works for a = b OR a = b because of equivalent disjunct optimization
SET join_algorithm = 'auto';
SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 }
-- works for a = b OR a = b because of equivalent disjunct optimization
SET join_algorithm = 'hash';
-- conditions for different table joined via OR
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError 403 }

View File

@ -17,7 +17,7 @@ SELECT uniqArray([1, 1, 2]),
toDayOfYear(d) % 2) toDayOfYear(d) % 2)
FROM numbers(100); FROM numbers(100);
SELECT repeat('a', number) SELECT repeat('aa', number)
FROM numbers(10e3) FROM numbers(10e3)
SETTINGS max_memory_usage=4e6, max_block_size=100 SETTINGS max_memory_usage=4e6, max_block_size=100
FORMAT Null; -- { serverError 241 } FORMAT Null; -- { serverError 241 }

View File

@ -0,0 +1,68 @@
join on OR chain (all left)
2 3
6 4
6 4
==
2 3
5 4
100 4
==
2 3 2 3
6 4 5 4
6 4 100 4
==
1
==
1 3
==
1 2 2 3
==
2 4
join on OR chain (all right)
2 3
6 4
6 4
==
2 3
5 4
100 4
==
2 3 2 3
6 4 5 4
6 4 100 4
==
0
1
==
0 4
1 3
==
0 0 6 4
1 2 2 3
==
2 4
6 5
join on OR chain (full)
2 3
6 4
6 4
==
2 3
5 4
100 4
==
2 3 2 3
6 4 5 4
6 4 100 4
==
0
1
==
0 4
1 3
==
0 0 6 4
1 2 2 3
==
2 4
6 5

View File

@ -0,0 +1,67 @@
SET joined_subquery_requires_alias = 0;
drop table if exists tab1;
drop table if exists tab2;
drop table if exists tab3;
create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1;
create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2;
create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3;
insert into tab1 values (1, 2);
insert into tab2 values (2, 3);
insert into tab2 values (6, 4);
insert into tab3 values (2, 3);
insert into tab3 values (5, 4);
insert into tab3 values (100, 4);
select 'join on OR chain (all left)';
select a2, b2 from tab2 all left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2;
select '==';
select a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3;
select '==';
select a2, b2, a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3;
select '==';
select a1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1;
select '==';
select a1, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2;
select '==';
select a1, b1, a2, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2;
select '==';
select a2, b2 + 1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1;
select 'join on OR chain (all right)';
select a2, b2 from tab2 all right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2;
select '==';
select a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3;
select '==';
select a2, b2, a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3;
select '==';
select a1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1;
select '==';
select a1, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2;
select '==';
select a1, b1, a2, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2;
select '==';
select a2, b2 + 1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1;
select 'join on OR chain (full)';
select a2, b2 from tab2 full join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2;
select '==';
select a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3;
select '==';
select a2, b2, a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3;
select '==';
select a1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1;
select '==';
select a1, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2;
select '==';
select a1, b1, a2, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2;
select '==';
select a2, b2 + 1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1;
drop table tab1;
drop table tab2;
drop table tab3;

View File

@ -0,0 +1,49 @@
join on OR chain (any left)
2 3
6 4
==
0 3
1 4
==
2 3 0 3
6 4 1 4
==
1
==
1 3
==
1 2 2 3
==
2 4
join on OR chain (any right)
2 3
6 4
6 4
==
2 3
5 4
100 4
==
2 3 2 3
6 4 5 4
6 4 100 4
==
0
1
==
0 4
1 3
==
0 0 6 4
1 2 2 3
==
2 4
6 5
any_join_distinct_right_table_keys = 1
0 0
2 3
6 4
==
2 3
5 4
100 4

View File

@ -0,0 +1,62 @@
SET joined_subquery_requires_alias = 0;
SET any_join_distinct_right_table_keys = 0;
drop table if exists tab1;
drop table if exists tab2;
drop table if exists tab3;
create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1;
create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2;
create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3;
insert into tab1 values (1, 2);
insert into tab2 values (2, 3);
insert into tab2 values (6, 4);
insert into tab3 values (2, 3);
insert into tab3 values (5, 4);
insert into tab3 values (100, 4);
select 'join on OR chain (any left)';
select a2, b2 from tab2 any left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2;
select '==';
select a3 == 5 OR a3 == 100, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3;
select '==';
select a2, b2, a3 == 5 OR a3 == 100, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, b3;
select '==';
select a1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1;
select '==';
select a1, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2;
select '==';
select a1, b1, a2, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2;
select '==';
select a2, b2 + 1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1;
select 'join on OR chain (any right)';
select a2, b2 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2;
select '==';
select a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3;
select '==';
select a2, b2, a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3;
select '==';
select a1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1;
select '==';
select a1, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2;
select '==';
select a1, b1, a2, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2;
select '==';
select a2, b2 + 1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1;
SET any_join_distinct_right_table_keys = 1;
select 'any_join_distinct_right_table_keys = 1';
select a2, b2 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2;
select '==';
select a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3;
drop table tab1;
drop table tab2;
drop table tab3;

View File

@ -0,0 +1,13 @@
any_join_distinct_right_table_keys = 0
2 3 2 3
6 4 5 4
==
2 3 2 3
6 4 5 4
any_join_distinct_right_table_keys = 1
2 3 2 3
6 4 5 4
==
2 3 2 3
6 4 5 4
1

View File

@ -0,0 +1,35 @@
SET joined_subquery_requires_alias = 0;
drop table if exists tab2;
drop table if exists tab3;
create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2;
create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3;
insert into tab2 values (2, 3);
insert into tab2 values (6, 4);
insert into tab2 values (998, 999);
insert into tab3 values (2, 3);
insert into tab3 values (5, 4);
insert into tab3 values (100, 4);
insert into tab3 values (1998, 1999);
set max_threads = 1;
SET any_join_distinct_right_table_keys = 0;
select 'any_join_distinct_right_table_keys = 0';
select tab2.*, tab3.* from tab2 any join tab3 on a2 = a3 or b2 = b3;
select '==';
select tab2.*, tab3.* from tab2 any join tab3 on b2 = b3 or a2 = a3;
SET any_join_distinct_right_table_keys = 1;
select 'any_join_distinct_right_table_keys = 1';
select tab2.*, tab3.* from tab2 any join tab3 on a2 = a3 or b2 = b3;
select '==';
select tab2.*, tab3.* from tab2 any join tab3 on b2 = b3 or a2 = a3;
SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb;
drop table tab2;
drop table tab3;

View File

@ -0,0 +1,10 @@
subqueries with OR
1
==
1
==
1 2
subquery column alias with OR
1 2 2 3
==
1 2 2 3

View File

@ -0,0 +1,27 @@
SET joined_subquery_requires_alias = 0;
SET max_threads = 1;
drop table if exists tab1;
drop table if exists tab2;
create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1;
create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2;
insert into tab1 values (1, 2);
insert into tab2 values (2, 3);
insert into tab2 values (6, 4);
select 'subqueries with OR';
select a1 from tab1 any left join (select * from tab2) on b1 = a2 or b2 = a1;
select '==';
select a1 from tab1 any left join (select a2, b2 from tab2) on b1 = a2 or b2 = a1;
select '==';
select a1, b1 from tab1 any left join (select * from tab2) on b1 = a2 or b2 = a1;
select 'subquery column alias with OR';
select a1, b1, a2, b2 from tab1 any left join (select *, a2 as z from tab2) on b1 + 1 = z + 1 or b1 = z * 2;
select '==';
select a1, b1, a2, b2 from tab1 any left join (select *, a2 + 1 as z from tab2) on b1 + 1 = z or b1 = z * 2;
drop table tab1;
drop table tab2;

View File

@ -0,0 +1,8 @@
a b c d e f a b c d e f
a b c d e f a b c d e f
a b c d e f a b c d e f
a b c d e f a b c d e f
a b c d e f a b c d e f
join on OR/AND chain
2 3 2 3
6 4 0 0

View File

@ -0,0 +1,35 @@
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.b = t2.b and t1.c = t2.b and t1.d = t2.b or t1.e = t2.e;
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b or t1.e = t2.e and t1.a=t2.a and t2.f=t1.f;
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f);
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.e = t2.e;
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.e = t2.e and t1.a=t2.a and t2.f=t1.f;
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b AND t1.e = t2.e OR t1.c = t2.b AND t1.e = t2.e OR t1.d = t2.b AND t1.f=t2.f OR t1.c = t2.b AND t1.f=t2.f;
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f));
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f);
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION }
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; -- { serverError INVALID_JOIN_ON_EXPRESSION }
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION }
select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f); -- { serverError INVALID_JOIN_ON_EXPRESSION }
SET joined_subquery_requires_alias = 0;
SET max_threads = 1;
drop table if exists tab2;
drop table if exists tab3;
create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2;
create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3;
insert into tab2 values (2, 3);
insert into tab2 values (6, 4);
insert into tab3 values (2, 3);
insert into tab3 values (5, 4);
insert into tab3 values (100, 4);
select 'join on OR/AND chain';
select a2, b2, a3, b3 from tab2 any left join tab3 on a2=a3 and a2 +1 = b3 + 0 or b2=b3 and a2 +1 = b3 + 0 ;
drop table tab2;
drop table tab3;

View File

@ -0,0 +1,3 @@
a b 42 a b 42
===
a b 42 a b 42

View File

@ -0,0 +1,3 @@
select * from (select 'a' as a, 'b' as b, 42 as forty_two) as t1 inner join (select 'a' as a, 'b' as b, 42 as forty_two) as t2 on t1.b = t2.a or t1.forty_two = t2.forty_two;
select '===';
select * from (select 'a' as a, 'b' as b, 42 as forty_two) as t1 inner join (select 'a' as a, 'b' as b, 42 as forty_two) as t2 on t1.b = t2.b or t1.forty_two = t2.forty_two;

View File

@ -0,0 +1,115 @@
1 left 1 2 1 2
5 left 1 2 1 2
5 left 1 2 1 2
5 left 1 2 1 2
5 left 1 2 1 2
5 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
15 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
16 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 left 1 2 1 2
17 any left 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 any right 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
17 full 1 2 1 2
555
a 0 a 0
a 0 a 1
a 1 a 0
a 1 a 1
a 0 a 0
a 0 a 1
a 1 a 0
a 1 a 1

View File

@ -0,0 +1,23 @@
select '1 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(1)) t2 on x = xx or y = yy;
select '5 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(5)) t2 on x = xx or y = yy;
select '15 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(15)) t2 on x = xx or y = yy;
select '16 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(16)) t2 on x = xx or y = yy;
select '17 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy;
select '17 any left', * from (select 1 as x, 2 as y) t1 any left join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy;
select '17 right', * from (select 1 as x, 2 as y) t1 right join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy;
select '17 any right', * from (select 1 as x, 2 as y) t1 any right join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy;
select '17 full', * from (select 1 as x, 2 as y) t1 full join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy;
select count(1) from (select * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(555)) t2 on x = xx or y = yy);
select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.c = t2.c or t1.a = t2.a order by t1.c, t2.c;
select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.a = t2.a or t1.c = t2.c order by t1.c, t2.c;

View File

@ -1,7 +1,7 @@
0 0 1 0 0 1
0 1 0 0 1 0
SELECT SELECT
isNull(id), id IS NULL,
`n.null`, `n.null`,
NOT `n.null` NOT `n.null`
FROM t_func_to_subcolumns FROM t_func_to_subcolumns
@ -31,7 +31,7 @@ FROM t_func_to_subcolumns
SELECT SELECT
id, id,
`n.null`, `n.null`,
isNull(right.n) right.n IS NULL
FROM t_func_to_subcolumns AS left FROM t_func_to_subcolumns AS left
ALL FULL OUTER JOIN ALL FULL OUTER JOIN
( (

View File

@ -1,108 +0,0 @@
-- hash_join --
--
222 2
222 222
333 333
--
222 222
333 333
--
222
333
--
1
1
1
1
1
1
1
1
1
--
2
2
3
2
3
2
3
2
3
2
3
2
3
2
3
2
3
--
222 2
333 3
222 2
333 3
--
0 2 AAA a
0 4 CCC CCC
1 111 111 0
2 222 2 0
2 222 222 2 AAA AAA
3 333 333 3 BBB BBB
--
2 222 2 2 AAA a
2 222 222 2 AAA AAA
-- partial_merge --
--
222 2
222 222
333 333
--
222 222
333 333
--
222
333
--
1
1
1
1
1
1
1
1
1
--
2
2
3
2
3
2
3
2
3
2
3
2
3
2
3
2
3
--
222 2
333 3
222 2
333 3
--
0 2 AAA a
0 4 CCC CCC
1 111 111 0
2 222 2 0
2 222 222 2 AAA AAA
3 333 333 3 BBB BBB
--
2 222 2 2 AAA a
2 222 222 2 AAA AAA

View File

@ -0,0 +1,73 @@
{%- macro jnull(join_use_nulls, value='') -%}
{#- default value or null if join_use_nulls is enabled -#}
{% if join_use_nulls == 1 %}\N{% else %}{{ value }}{% endif %}
{%- endmacro -%}
{% for jn in [0, 1] -%}
-- hash_join --
--
222 2
222 222
333 333
--
222 222
333 333
--
222
333
--
1
1
1
1
1
1
1
1
1
--
2
2
3
2
3
2
3
2
3
2
3
2
3
2
3
2
3
--
222 2
333 3
222 2
333 3
--
{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 AAA a
{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 CCC CCC
1 111 111 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }}
2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }}
2 222 222 2 AAA AAA
3 333 333 3 BBB BBB
{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 101 CCC CCC
1 111 111 2 1 AAA AAA
2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }}
2 222 222 2 1 AAA AAA
3 333 333 2 3 AAA a
3 333 333 3 100 BBB BBB
2 222 2 2 AAA AAA
2 222 222 2 AAA AAA
3 333 333 3 BBB BBB
--
2 222 2 2 AAA a
2 222 222 2 AAA AAA
t22 1 111 111 2 1 AAA AAA
t22 1 111 111 2 1 AAA AAA
t22 1 111 111 2 1 AAA AAA
t22 1 111 111 2 1 AAA AAA
{% endfor -%}

View File

@ -0,0 +1,102 @@
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t2_nullable;
DROP TABLE IF EXISTS t2_lc;
DROP TABLE IF EXISTS t22;
CREATE TABLE t1 (`id` Int32, key String, key2 String) ENGINE = TinyLog;
CREATE TABLE t2 (`id` Int32, key String, key2 String) ENGINE = TinyLog;
CREATE TABLE t2_nullable (`id` Int32, key String, key2 Nullable(String)) ENGINE = TinyLog;
CREATE TABLE t2_lc (`id` Int32, key String, key2 LowCardinality(String)) ENGINE = TinyLog;
CREATE TABLE t22 (`id` Int32, `idd` Int32, `key` String, `key2` String) ENGINE = TinyLog;
INSERT INTO t1 VALUES (1, '111', '111'),(2, '222', '2'),(2, '222', '222'),(3, '333', '333');
INSERT INTO t2 VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC');
INSERT INTO t2_nullable VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', NULL),(4, 'CCC', 'CCC');
INSERT INTO t2_lc VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC');
INSERT INTO t22 VALUES (2, 1, 'AAA', 'AAA'),(2, 3, 'AAA', 'a'),(3, 100, 'BBB', 'BBB'),(4, 101, 'CCC', 'CCC');
{% for join_use_nulls in [0, 1] -%}
SET join_use_nulls = {{ join_use_nulls }};
SET join_algorithm = 'hash';
SELECT '-- hash_join --';
SELECT '--';
SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2;
SELECT '--';
SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2;
SELECT '--';
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2;
SELECT '--';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2;
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3;
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2);
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t1.key);
-- DISTINCT is used to remove the difference between 'hash' and 'merge' join: 'merge' doesn't support `any_join_distinct_right_table_keys`
SELECT '--';
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t2.key2 != '';
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t2.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t2.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t2.key2 != ''));
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t2.key2 != ''));
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t1.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t1.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t1.key2 != ''));
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t1.key2 != ''));
SELECT '--';
SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e;
-- `e + 1` is UInt16
SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e + 1; -- { serverError 403 }
SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1);
SELECT '--';
SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST;
SELECT t1.id, t1.key, t1.key2, t22.id, t22.idd, t22.key, t22.key2 FROM t1 FULL JOIN t22 ON t1.id == t22.id AND t22.key == t22.key2 AND t1.key == t1.key2 OR t1.id = t22.idd AND t1.key = t1.key2 ORDER BY t1.id NULLS FIRST, t22.id NULLS FIRST;
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id;
-- non-equi condition containing columns from different tables doesn't supported yet
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 }
SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 }
SELECT '--';
-- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key
SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333';
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 }
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id;
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2;
SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd;
SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd;
{% endfor -%}
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t2_nullable;
DROP TABLE IF EXISTS t2_lc;
DROP TABLE IF EXISTS t22;

View File

@ -0,0 +1,60 @@
{%- macro jnull(join_use_nulls, value='') -%}
{#- default value or null if join_use_nulls is enabled -#}
{% if join_use_nulls == 1 %}\N{% else %}{{ value }}{% endif %}
{%- endmacro -%}
{% for jn in [0, 1] -%}
-- partial_merge --
--
222 2
222 222
333 333
--
222 222
333 333
--
222
333
--
1
1
1
1
1
1
1
1
1
--
2
2
3
2
3
2
3
2
3
2
3
2
3
2
3
2
3
--
222 2
333 3
222 2
333 3
--
{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 AAA a
{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 CCC CCC
1 111 111 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }}
2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }}
2 222 222 2 AAA AAA
3 333 333 3 BBB BBB
--
2 222 2 2 AAA a
2 222 222 2 AAA AAA
{% endfor -%}

View File

@ -2,76 +2,22 @@ DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t2_nullable; DROP TABLE IF EXISTS t2_nullable;
DROP TABLE IF EXISTS t2_lc; DROP TABLE IF EXISTS t2_lc;
DROP TABLE IF EXISTS t22;
CREATE TABLE t1 (`id` Int32, key String, key2 String) ENGINE = TinyLog; CREATE TABLE t1 (`id` Int32, key String, key2 String) ENGINE = TinyLog;
CREATE TABLE t2 (`id` Int32, key String, key2 String) ENGINE = TinyLog; CREATE TABLE t2 (`id` Int32, key String, key2 String) ENGINE = TinyLog;
CREATE TABLE t2_nullable (`id` Int32, key String, key2 Nullable(String)) ENGINE = TinyLog; CREATE TABLE t2_nullable (`id` Int32, key String, key2 Nullable(String)) ENGINE = TinyLog;
CREATE TABLE t2_lc (`id` Int32, key String, key2 LowCardinality(String)) ENGINE = TinyLog; CREATE TABLE t2_lc (`id` Int32, key String, key2 LowCardinality(String)) ENGINE = TinyLog;
CREATE TABLE t22 (`id` Int32, `idd` Int32, `key` String, `key2` String) ENGINE = TinyLog;
INSERT INTO t1 VALUES (1, '111', '111'),(2, '222', '2'),(2, '222', '222'),(3, '333', '333'); INSERT INTO t1 VALUES (1, '111', '111'),(2, '222', '2'),(2, '222', '222'),(3, '333', '333');
INSERT INTO t2 VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC'); INSERT INTO t2 VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC');
INSERT INTO t2_nullable VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', NULL),(4, 'CCC', 'CCC'); INSERT INTO t2_nullable VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', NULL),(4, 'CCC', 'CCC');
INSERT INTO t2_lc VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC'); INSERT INTO t2_lc VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC');
INSERT INTO t22 VALUES (2, 1, 'AAA', 'AAA'),(2, 3, 'AAA', 'a'),(3, 100, 'BBB', 'BBB'),(4, 101, 'CCC', 'CCC');
SELECT '-- hash_join --'; {% for join_use_nulls in [0, 1] -%}
SELECT '--'; SET join_use_nulls = {{ join_use_nulls }};
SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2;
SELECT '--';
SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2;
SELECT '--';
SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2;
SELECT '--';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2;
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3;
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2);
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%';
SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t1.key);
-- DISTINCT is used to remove the difference between 'hash' and 'merge' join: 'merge' doesn't support `any_join_distinct_right_table_keys`
SELECT '--';
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t2.key2 != '';
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t2.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t2.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t2.key2 != ''));
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t2.key2 != ''));
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t1.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t1.key2 != '');
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t1.key2 != ''));
SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t1.key2 != ''));
SELECT '--';
SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e;
-- `e + 1` is UInt16
SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e + 1; -- { serverError 403 }
SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1);
SELECT '--';
SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id, t2.id;
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 }
-- non-equi condition containing columns from different tables doesn't supported yet
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 }
SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 }
SELECT '--';
-- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key
SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333';
SET join_algorithm = 'partial_merge'; SET join_algorithm = 'partial_merge';
@ -116,7 +62,7 @@ SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id =
SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1); SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1);
SELECT '--'; SELECT '--';
SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id, t2.id; SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST;
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 }
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 }
@ -127,6 +73,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 }
SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- { serverError 48 }
-- non-equi condition containing columns from different tables doesn't supported yet -- non-equi condition containing columns from different tables doesn't supported yet
SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 }
SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 }
@ -135,7 +82,11 @@ SELECT '--';
-- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key
SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333';
{% endfor -%}
DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2;
DROP TABLE IF EXISTS t2_nullable; DROP TABLE IF EXISTS t2_nullable;
DROP TABLE IF EXISTS t2_lc; DROP TABLE IF EXISTS t2_lc;
DROP TABLE IF EXISTS t22;

View File

@ -0,0 +1,104 @@
Map full text bloom filter tokenbf mapKeys
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value
Map fixed full text bloom filter tokenbf mapKeys
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value
Map full text bloom filter tokenbf mapValues
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value
Map fixed full text bloom filter tokenbf mapKeys
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value
Map full text bloom filter ngrambf mapKeys
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value
Map fixed full text bloom filter ngrambf mapKeys
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value
Map full text bloom filter ngrambf mapValues
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value
Map fixed full text bloom filter ngrambf mapKeys
Equals with existing key
0 {'K0':'V0'} {'K0':'V0'}
Equals with non existing key
Equals with non existing key and default value
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with existing key
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key
0 {'K0':'V0'} {'K0':'V0'}
1 {'K1':'V1'} {'K1':'V1'}
Not equals with non existing key and default value

View File

@ -0,0 +1,174 @@
DROP TABLE IF EXISTS bf_tokenbf_map_keys_test;
DROP TABLE IF EXISTS bf_ngrambf_map_keys_test;
CREATE TABLE bf_tokenbf_map_keys_test
(
row_id UInt32,
map Map(String, String),
map_fixed Map(FixedString(2), String),
INDEX map_keys_tokenbf mapKeys(map) TYPE tokenbf_v1(256,2,0) GRANULARITY 1,
INDEX map_fixed_keys_tokenbf mapKeys(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1
) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1;
INSERT INTO bf_tokenbf_map_keys_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'});
SELECT 'Map full text bloom filter tokenbf mapKeys';
SELECT 'Equals with existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_keys_tokenbf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_keys_tokenbf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_keys_tokenbf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_keys_tokenbf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K3'] != '';
SELECT 'Map fixed full text bloom filter tokenbf mapKeys';
SELECT 'Equals with existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K3'] != '';
DROP TABLE bf_tokenbf_map_keys_test;
CREATE TABLE bf_tokenbf_map_values_test
(
row_id UInt32,
map Map(String, String),
map_fixed Map(FixedString(2), String),
INDEX map_values_tokenbf mapValues(map) TYPE tokenbf_v1(256,2,0) GRANULARITY 1,
INDEX map_fixed_values_tokenbf mapValues(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1
) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1;
INSERT INTO bf_tokenbf_map_values_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'});
SELECT 'Map full text bloom filter tokenbf mapValues';
SELECT 'Equals with existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_values_tokenbf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_values_tokenbf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_values_test WHERE map['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_values_tokenbf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_values_tokenbf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_values_test WHERE map['K3'] != '';
SELECT 'Map fixed full text bloom filter tokenbf mapKeys';
SELECT 'Equals with existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K3'] != '';
DROP TABLE bf_tokenbf_map_values_test;
CREATE TABLE bf_ngrambf_map_keys_test
(
row_id UInt32,
map Map(String, String),
map_fixed Map(FixedString(2), String),
INDEX map_keys_ngrambf mapKeys(map) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1,
INDEX map_fixed_keys_ngrambf mapKeys(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1
) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1;
INSERT INTO bf_ngrambf_map_keys_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'});
SELECT 'Map full text bloom filter ngrambf mapKeys';
SELECT 'Equals with existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_keys_ngrambf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_keys_ngrambf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_keys_ngrambf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_keys_ngrambf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K3'] != '';
SELECT 'Map fixed full text bloom filter ngrambf mapKeys';
SELECT 'Equals with existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K3'] != '';
DROP TABLE bf_ngrambf_map_keys_test;
CREATE TABLE bf_ngrambf_map_values_test
(
row_id UInt32,
map Map(String, String),
map_fixed Map(FixedString(2), String),
INDEX map_values_ngrambf mapKeys(map) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1,
INDEX map_fixed_values_ngrambf mapKeys(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1
) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1;
INSERT INTO bf_ngrambf_map_values_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'});
SELECT 'Map full text bloom filter ngrambf mapValues';
SELECT 'Equals with existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_values_ngrambf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_values_ngrambf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_values_test WHERE map['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_values_ngrambf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_values_ngrambf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_values_test WHERE map['K3'] != '';
SELECT 'Map fixed full text bloom filter ngrambf mapKeys';
SELECT 'Equals with existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf';
SELECT 'Equals with non existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf';
SELECT 'Equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K3'] = '';
SELECT 'Not equals with existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf';
SELECT 'Not equals with non existing key';
SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf';
SELECT 'Not equals with non existing key and default value';
SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K3'] != '';
DROP TABLE bf_ngrambf_map_values_test;

View File

@ -55,7 +55,7 @@ export -f insert1
export -f insert2 export -f insert2
export -f select1 export -f select1
export -f select2 export -f select2
# export -f truncate1 export -f truncate1
for _ in {1..5}; do for _ in {1..5}; do
timeout $TIMEOUT bash -c insert1 & timeout $TIMEOUT bash -c insert1 &
@ -64,7 +64,7 @@ done
timeout $TIMEOUT bash -c select1 & timeout $TIMEOUT bash -c select1 &
timeout $TIMEOUT bash -c select2 & timeout $TIMEOUT bash -c select2 &
# timeout $TIMEOUT bash -c truncate1 & timeout $TIMEOUT bash -c truncate1 &
wait wait
echo "OK" echo "OK"

View File

@ -0,0 +1,64 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1
1

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