Merge branch 'master' into revert-62508-revert-62123-full-syntax-highlight

This commit is contained in:
Alexey Milovidov 2024-04-12 00:44:02 +02:00
commit da05238c8e
83 changed files with 2569 additions and 712 deletions

View File

@ -23,10 +23,10 @@ jobs:
clear-repository: true # to ensure correct digests
fetch-depth: 0 # to get version
filter: tree:0
- name: Check sync PR
- name: Merge sync PR
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 sync_pr.py || :
python3 sync_pr.py --merge || :
- name: Python unit tests
run: |
cd "$GITHUB_WORKSPACE/tests/ci"

View File

@ -157,16 +157,25 @@ jobs:
################################# Stage Final #################################
#
FinishCheck:
if: ${{ !failure() && !cancelled() && github.event_name != 'merge_group' }}
needs: [Tests_1, Tests_2]
if: ${{ !failure() && !cancelled() }}
needs: [Tests_1, Tests_2, Builds_1_Report, Builds_2_Report]
runs-on: [self-hosted, style-checker]
steps:
- name: Check out repository code
uses: ClickHouse/checkout@v1
- name: Check sync status
if: ${{ github.event_name == 'merge_group' }}
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 sync_pr.py --status
- name: Finish label
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 finish_check.py
- name: Auto merge if approved
if: ${{ github.event_name != 'merge_group' }}
run: |
cd "$GITHUB_WORKSPACE/tests/ci"
python3 merge_pr.py --check-approved

View File

@ -26,6 +26,11 @@
<table_function_remote_max_addresses>
<max>200</max>
</table_function_remote_max_addresses>
<!-- Don't waste cycles testing the old interpreter. Spend time in the new analyzer instead -->
<allow_experimental_analyzer>
<readonly/>
</allow_experimental_analyzer>
</constraints>
</default>
</profiles>

View File

@ -42,6 +42,19 @@ Type: UInt32
Default: 1
## auth_use_forwarded_address
Use originating address for authentication for clients connected through proxy.
:::note
This setting should be used with extra caution since forwarded address can be easily spoofed - server accepting such authentication should not be accessed directly but rather exclusively through a trusted proxy.
:::
Type: Bool
Default: 0
## background_buffer_flush_schedule_pool_size
The maximum number of threads that will be used for performing flush operations for Buffer-engine tables in the background.

View File

@ -0,0 +1,155 @@
---
slug: /en/operations/settings/composable-protocols
sidebar_position: 64
sidebar_label: Composable Protocols
---
# Composable Protocols
Composable protocols allows more flexible configuration of TCP access to the ClickHouse server. This configuration can co-exist with or replace conventional configuration.
## Composable protocols section is denoted as `protocols` in configuration xml
**Example:**
``` xml
<protocols>
</protocols>
```
## Basic modules define protocol layers
**Example:**
``` xml
<protocols>
<!-- plain_http module -->
<plain_http>
<type>http</type>
</plain_http>
</protocols>
```
where:
- `plain_http` - name which can be referred by another layer
- `type` - denotes protocol handler which will be instantiated to process data, set of protocol handlers is predefined:
* `tcp` - native clickhouse protocol handler
* `http` - http clickhouse protocol handler
* `tls` - TLS encryption layer
* `proxy1` - PROXYv1 layer
* `mysql` - MySQL compatibility protocol handler
* `postgres` - PostgreSQL compatibility protocol handler
* `prometheus` - Prometheus protocol handler
* `interserver` - clickhouse interserver handler
:::note
`gRPC` protocol handler is not implemented for `Composable protocols`
:::
## Endpoint (i.e. listening port) is denoted by `<port>` and (optional) `<host>` tags
**Example:**
``` xml
<protocols>
<plain_http>
<type>http</type>
<!-- endpoint -->
<host>127.0.0.1</host>
<port>8123</port>
</plain_http>
</protocols>
```
If `<host>` is omitted, then `<listen_host>` from root config is used.
## Layers sequence is defined by `<impl>` tag, referencing another module
**Example:** definition for HTTPS protocol
``` xml
<protocols>
<!-- http module -->
<plain_http>
<type>http</type>
</plain_http>
<!-- https module configured as a tls layer on top of plain_http module -->
<https>
<type>tls</type>
<impl>plain_http</impl>
<host>127.0.0.1</host>
<port>8443</port>
</https>
</protocols>
```
## Endpoint can be attached to any layer
**Example:** definition for HTTP (port 8123) and HTTPS (port 8443) endpoints
``` xml
<protocols>
<plain_http>
<type>http</type>
<host>127.0.0.1</host>
<port>8123</port>
</plain_http>
<https>
<type>tls</type>
<impl>plain_http</impl>
<host>127.0.0.1</host>
<port>8443</port>
</https>
</protocols>
```
## Additional endpoints can be defined by referencing any module and omitting `<type>` tag
**Example:** `another_http` endpoint is defined for `plain_http` module
``` xml
<protocols>
<plain_http>
<type>http</type>
<host>127.0.0.1</host>
<port>8123</port>
</plain_http>
<https>
<type>tls</type>
<impl>plain_http</impl>
<host>127.0.0.1</host>
<port>8443</port>
</https>
<another_http>
<impl>plain_http</impl>
<host>127.0.0.1</host>
<port>8223</port>
</another_http>
</protocols>
```
## Some modules can contain specific for its layer parameters
**Example:** for TLS layer private key (`privateKeyFile`) and certificate files (`certificateFile`) can be specified
``` xml
<protocols>
<plain_http>
<type>http</type>
<host>127.0.0.1</host>
<port>8123</port>
</plain_http>
<https>
<type>tls</type>
<impl>plain_http</impl>
<host>127.0.0.1</host>
<port>8443</port>
<privateKeyFile>another_server.key</privateKeyFile>
<certificateFile>another_server.crt</certificateFile>
</https>
</protocols>
```

View File

@ -9,6 +9,7 @@ Columns:
- `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query.
- `event_date` ([Date](../../sql-reference/data-types/date.md)) — Date of the entry.
- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time of the entry.
- `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Time of the entry with microseconds precision.
- `id` ([String](../../sql-reference/data-types/string.md)) — Identifier of the backup or restore operation.
- `name` ([String](../../sql-reference/data-types/string.md)) — Name of the backup storage (the contents of the `FROM` or `TO` clause).
@ -67,6 +68,7 @@ Row 2:
──────
hostname: clickhouse.eu-central1.internal
event_date: 2023-08-19
event_time: 2023-08-19 11:08:56
event_time_microseconds: 2023-08-19 11:08:56.916192
id: e5b74ecb-f6f1-426a-80be-872f90043885
name: Disk('backups_disk', '1.zip')

View File

@ -970,7 +970,7 @@ If the haystack or the LIKE expression are not valid UTF-8, the behavior is unde
No automatic Unicode normalization is performed, you can use the [normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
To match against literal `%`, `_` and `/` (which are LIKE metacharacters), prepend them with a backslash: `\%`, `\_` and `\\`.
To match against literal `%`, `_` and `\` (which are LIKE metacharacters), prepend them with a backslash: `\%`, `\_` and `\\`.
The backslash loses its special meaning (i.e. is interpreted literally) if it prepends a character different than `%`, `_` or `\`.
Note that ClickHouse requires backslashes in strings [to be quoted as well](../syntax.md#string), so you would actually need to write `\\%`, `\\_` and `\\\\`.
@ -1768,4 +1768,4 @@ SELECT hasTokenCaseInsensitiveOrNull('Hello World','hello,world');
```response
null
```
```

View File

@ -133,8 +133,6 @@ For the query to run successfully, the following conditions must be met:
- Both tables must have the same indices and projections.
- Both tables must have the same storage policy.
If both tables have the same storage policy, use hardlink to attach partition. Otherwise, use copying the data to attach partition.
## REPLACE PARTITION
``` sql

View File

@ -482,6 +482,7 @@ void Client::connect()
server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch);
load_suggestions = is_interactive && (server_revision >= Suggest::MIN_SERVER_REVISION) && !config().getBool("disable_suggestion", false);
wait_for_suggestions_to_load = config().getBool("wait_for_suggestions_to_load", false);
if (server_display_name = connection->getServerDisplayName(connection_parameters.timeouts); server_display_name.empty())
server_display_name = config().getString("host", "localhost");
@ -687,7 +688,11 @@ bool Client::processWithFuzzing(const String & full_query)
try
{
const char * begin = full_query.data();
orig_ast = parseQuery(begin, begin + full_query.size(), true);
orig_ast = parseQuery(begin, begin + full_query.size(),
global_context->getSettingsRef(),
/*allow_multi_statements=*/ true,
/*is_interactive=*/ is_interactive,
/*ignore_error=*/ ignore_error);
}
catch (const Exception & e)
{

View File

@ -46,12 +46,12 @@ INCBIN(resource_users_xml, SOURCE_DIR "/programs/server/users.xml");
*
* The following steps are performed:
*
* - copying the binary to binary directory (/usr/bin).
* - copying the binary to binary directory (/usr/bin/)
* - creation of symlinks for tools.
* - creation of clickhouse user and group.
* - creation of config directory (/etc/clickhouse-server).
* - creation of config directory (/etc/clickhouse-server/).
* - creation of default configuration files.
* - creation of a directory for logs (/var/log/clickhouse-server).
* - creation of a directory for logs (/var/log/clickhouse-server/).
* - creation of a data directory if not exists.
* - setting a password for default user.
* - choose an option to listen connections.
@ -226,7 +226,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv)
desc.add_options()
("help,h", "produce help message")
("prefix", po::value<std::string>()->default_value("/"), "prefix for all paths")
#if defined (OS_DARWIN)
/// https://stackoverflow.com/a/36734569/22422288
("binary-path", po::value<std::string>()->default_value("usr/local/bin"), "where to install binaries")
#else
("binary-path", po::value<std::string>()->default_value("usr/bin"), "where to install binaries")
#endif
("config-path", po::value<std::string>()->default_value("etc/clickhouse-server"), "where to install configs")
("log-path", po::value<std::string>()->default_value("var/log/clickhouse-server"), "where to create log directory")
("data-path", po::value<std::string>()->default_value("var/lib/clickhouse"), "directory for data")
@ -1216,7 +1221,12 @@ int mainEntryClickHouseStart(int argc, char ** argv)
desc.add_options()
("help,h", "produce help message")
("prefix", po::value<std::string>()->default_value("/"), "prefix for all paths")
#if defined (OS_DARWIN)
/// https://stackoverflow.com/a/36734569/22422288
("binary-path", po::value<std::string>()->default_value("usr/local/bin"), "directory with binary")
#else
("binary-path", po::value<std::string>()->default_value("usr/bin"), "directory with binary")
#endif
("config-path", po::value<std::string>()->default_value("etc/clickhouse-server"), "directory with configs")
("pid-path", po::value<std::string>()->default_value("var/run/clickhouse-server"), "directory for pid file")
("user", po::value<std::string>()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user")
@ -1332,7 +1342,12 @@ int mainEntryClickHouseRestart(int argc, char ** argv)
desc.add_options()
("help,h", "produce help message")
("prefix", po::value<std::string>()->default_value("/"), "prefix for all paths")
#if defined (OS_DARWIN)
/// https://stackoverflow.com/a/36734569/22422288
("binary-path", po::value<std::string>()->default_value("usr/local/bin"), "directory with binary")
#else
("binary-path", po::value<std::string>()->default_value("usr/bin"), "directory with binary")
#endif
("config-path", po::value<std::string>()->default_value("etc/clickhouse-server"), "directory with configs")
("pid-path", po::value<std::string>()->default_value("var/run/clickhouse-server"), "directory for pid file")
("user", po::value<std::string>()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user")

View File

@ -413,8 +413,20 @@ void LocalServer::setupUsers()
void LocalServer::connect()
{
connection_parameters = ConnectionParameters(config(), "localhost");
ReadBuffer * in;
auto table_file = config().getString("table-file", "-");
if (table_file == "-" || table_file == "stdin")
{
in = &std_in;
}
else
{
input = std::make_unique<ReadBufferFromFile>(table_file);
in = input.get();
}
connection = LocalConnection::createConnection(
connection_parameters, global_context, need_render_progress, need_render_profile_events, server_display_name);
connection_parameters, global_context, in, need_render_progress, need_render_profile_events, server_display_name);
}
@ -560,6 +572,7 @@ void LocalServer::processConfig()
const std::string clickhouse_dialect{"clickhouse"};
load_suggestions = (is_interactive || delayed_interactive) && !config().getBool("disable_suggestion", false)
&& config().getString("dialect", clickhouse_dialect) == clickhouse_dialect;
wait_for_suggestions_to_load = config().getBool("wait_for_suggestions_to_load", false);
auto logging = (config().has("logger.console")
|| config().has("logger.level")
@ -835,6 +848,8 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp
config().setString("logger.level", options["logger.level"].as<std::string>());
if (options.count("send_logs_level"))
config().setString("send_logs_level", options["send_logs_level"].as<std::string>());
if (options.count("wait_for_suggestions_to_load"))
config().setBool("wait_for_suggestions_to_load", true);
}
void LocalServer::readArguments(int argc, char ** argv, Arguments & common_arguments, std::vector<Arguments> &, std::vector<Arguments> &)

View File

@ -65,6 +65,8 @@ private:
std::optional<StatusFile> status;
std::optional<std::filesystem::path> temporary_directory_to_delete;
std::unique_ptr<ReadBufferFromFile> input;
};
}

View File

@ -15,6 +15,7 @@
#include <Functions/logical.h>
#include <Common/logger_useful.h>
#include <Analyzer/Utils.h>
namespace DB
@ -61,47 +62,7 @@ const QueryTreeNodePtr & getEquiArgument(const QueryTreeNodePtr & cond, size_t i
return func->getArguments().getNodes()[index];
}
/// Check that node has only one source and return it.
/// {_, false} - multiple sources
/// {nullptr, true} - no sources
/// {source, true} - single source
std::pair<const IQueryTreeNode *, bool> getExpressionSource(const QueryTreeNodePtr & node)
{
if (const auto * column = node->as<ColumnNode>())
{
auto source = column->getColumnSourceOrNull();
if (!source)
return {nullptr, false};
return {source.get(), true};
}
if (const auto * func = node->as<FunctionNode>())
{
const IQueryTreeNode * source = nullptr;
const auto & args = func->getArguments().getNodes();
for (const auto & arg : args)
{
auto [arg_source, is_ok] = getExpressionSource(arg);
if (!is_ok)
return {nullptr, false};
if (!source)
source = arg_source;
else if (arg_source && !source->isEqual(*arg_source))
return {nullptr, false};
}
return {source, true};
}
if (node->as<ConstantNode>())
return {nullptr, true};
return {nullptr, false};
}
bool findInTableExpression(const IQueryTreeNode * source, const QueryTreeNodePtr & table_expression)
bool findInTableExpression(const QueryTreeNodePtr & source, const QueryTreeNodePtr & table_expression)
{
if (!source)
return true;
@ -115,7 +76,6 @@ bool findInTableExpression(const IQueryTreeNode * source, const QueryTreeNodePtr
|| findInTableExpression(source, join_node->getRightTableExpression());
}
return false;
}
@ -169,10 +129,10 @@ public:
auto left_src = getExpressionSource(lhs_equi_argument);
auto right_src = getExpressionSource(rhs_equi_argument);
if (left_src.second && right_src.second && left_src.first && right_src.first)
if (left_src && right_src)
{
if ((findInTableExpression(left_src.first, left_table) && findInTableExpression(right_src.first, right_table)) ||
(findInTableExpression(left_src.first, right_table) && findInTableExpression(right_src.first, left_table)))
if ((findInTableExpression(left_src, left_table) && findInTableExpression(right_src, right_table)) ||
(findInTableExpression(left_src, right_table) && findInTableExpression(right_src, left_table)))
{
can_convert_cross_to_inner = true;
continue;

View File

@ -25,8 +25,9 @@ class JoinOnLogicalExpressionOptimizerVisitor : public InDepthQueryTreeVisitorWi
public:
using Base = InDepthQueryTreeVisitorWithContext<JoinOnLogicalExpressionOptimizerVisitor>;
explicit JoinOnLogicalExpressionOptimizerVisitor(ContextPtr context)
explicit JoinOnLogicalExpressionOptimizerVisitor(const JoinNode * join_node_, ContextPtr context)
: Base(std::move(context))
, join_node(join_node_)
{}
void enterImpl(QueryTreeNodePtr & node)
@ -55,10 +56,11 @@ public:
}
private:
const JoinNode * join_node;
bool need_rerun_resolve = false;
/// Returns true if type of some operand is changed and parent function needs to be re-resolved
static bool tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node, const ContextPtr & context)
bool tryOptimizeIsNotDistinctOrIsNull(QueryTreeNodePtr & node, const ContextPtr & context)
{
auto & function_node = node->as<FunctionNode &>();
chassert(function_node.getFunctionName() == "or");
@ -93,6 +95,21 @@ private:
const auto & func_name = argument_function->getFunctionName();
if (func_name == "equals" || func_name == "isNotDistinctFrom")
{
const auto & argument_nodes = argument_function->getArguments().getNodes();
if (argument_nodes.size() != 2)
continue;
/// We can rewrite to a <=> b only if we are joining on a and b,
/// because the function is not yet implemented for other cases.
auto first_src = getExpressionSource(argument_nodes[0]);
auto second_src = getExpressionSource(argument_nodes[1]);
if (!first_src || !second_src)
continue;
const auto & lhs_join = *join_node->getLeftTableExpression();
const auto & rhs_join = *join_node->getRightTableExpression();
bool arguments_from_both_sides = (first_src->isEqual(lhs_join) && second_src->isEqual(rhs_join)) ||
(first_src->isEqual(rhs_join) && second_src->isEqual(lhs_join));
if (!arguments_from_both_sides)
continue;
equals_functions_indices.push_back(or_operands.size() - 1);
}
else if (func_name == "and")
@ -231,7 +248,7 @@ public:
/// Operator <=> is not supported outside of JOIN ON section
if (join_node->hasJoinExpression())
{
JoinOnLogicalExpressionOptimizerVisitor join_on_visitor(getContext());
JoinOnLogicalExpressionOptimizerVisitor join_on_visitor(join_node, getContext());
join_on_visitor.visit(join_node->getJoinExpression());
}
return;

View File

@ -760,6 +760,54 @@ QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_ty
return function_node;
}
/** Returns:
* {_, false} - multiple sources
* {nullptr, true} - no sources (for constants)
* {source, true} - single source
*/
std::pair<QueryTreeNodePtr, bool> getExpressionSourceImpl(const QueryTreeNodePtr & node)
{
if (const auto * column = node->as<ColumnNode>())
{
auto source = column->getColumnSourceOrNull();
if (!source)
return {nullptr, false};
return {source, true};
}
if (const auto * func = node->as<FunctionNode>())
{
QueryTreeNodePtr source = nullptr;
const auto & args = func->getArguments().getNodes();
for (const auto & arg : args)
{
auto [arg_source, is_ok] = getExpressionSourceImpl(arg);
if (!is_ok)
return {nullptr, false};
if (!source)
source = arg_source;
else if (arg_source && !source->isEqual(*arg_source))
return {nullptr, false};
}
return {source, true};
}
if (node->as<ConstantNode>())
return {nullptr, true};
return {nullptr, false};
}
QueryTreeNodePtr getExpressionSource(const QueryTreeNodePtr & node)
{
auto [source, is_ok] = getExpressionSourceImpl(node);
if (!is_ok)
return nullptr;
return source;
}
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context)
{
const auto & storage_snapshot = table_node->as<TableNode>()->getStorageSnapshot();

View File

@ -105,6 +105,9 @@ NameSet collectIdentifiersFullNames(const QueryTreeNodePtr & node);
/// Wrap node into `_CAST` function
QueryTreeNodePtr createCastFunction(QueryTreeNodePtr node, DataTypePtr result_type, ContextPtr context);
/// Checks that node has only one source and returns it
QueryTreeNodePtr getExpressionSource(const QueryTreeNodePtr & node);
/// Build subquery which we execute for `IN table` function.
QueryTreeNodePtr buildSubqueryToReadColumnsFromTableExpression(QueryTreeNodePtr table_node, const ContextPtr & context);

View File

@ -125,7 +125,7 @@ BackupEntries BackupEntriesCollector::run()
= BackupSettings::Util::filterHostIDs(backup_settings.cluster_host_ids, backup_settings.shard_num, backup_settings.replica_num);
/// Do renaming in the create queries according to the renaming config.
renaming_map = makeRenamingMapFromBackupQuery(backup_query_elements);
renaming_map = BackupUtils::makeRenamingMap(backup_query_elements);
/// Calculate the root path for collecting backup entries, it's either empty or has the format "shards/<shard_num>/replicas/<replica_num>/".
calculateRootPathInBackup();
@ -570,17 +570,16 @@ std::vector<std::pair<ASTPtr, StoragePtr>> BackupEntriesCollector::findTablesInD
checkIsQueryCancelled();
auto filter_by_table_name = [my_database_info = &database_info](const String & table_name)
auto filter_by_table_name = [&](const String & table_name)
{
/// We skip inner tables of materialized views.
if (table_name.starts_with(".inner_id."))
if (BackupUtils::isInnerTable(database_name, table_name))
return false;
if (my_database_info->tables.contains(table_name))
if (database_info.tables.contains(table_name))
return true;
if (my_database_info->all_tables)
return !my_database_info->except_table_names.contains(table_name);
if (database_info.all_tables)
return !database_info.except_table_names.contains(table_name);
return false;
};

View File

@ -8,10 +8,10 @@
#include <Common/setThreadName.h>
namespace DB
namespace DB::BackupUtils
{
DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & elements)
DDLRenamingMap makeRenamingMap(const ASTBackupQuery::Elements & elements)
{
DDLRenamingMap map;
@ -120,4 +120,15 @@ bool compareRestoredDatabaseDef(const IAST & restored_database_create_query, con
return compareRestoredTableDef(restored_database_create_query, create_query_from_backup, global_context);
}
bool isInnerTable(const QualifiedTableName & table_name)
{
return isInnerTable(table_name.database, table_name.table);
}
bool isInnerTable(const String & /* database_name */, const String & table_name)
{
/// We skip inner tables of materialized views.
return table_name.starts_with(".inner.") || table_name.starts_with(".inner_id.");
}
}

View File

@ -9,9 +9,13 @@ namespace DB
class IBackup;
class AccessRightsElements;
class DDLRenamingMap;
struct QualifiedTableName;
namespace BackupUtils
{
/// Initializes a DDLRenamingMap from a BACKUP or RESTORE query.
DDLRenamingMap makeRenamingMapFromBackupQuery(const ASTBackupQuery::Elements & elements);
DDLRenamingMap makeRenamingMap(const ASTBackupQuery::Elements & elements);
/// Returns access required to execute BACKUP query.
AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements & elements);
@ -20,4 +24,10 @@ AccessRightsElements getRequiredAccessToBackup(const ASTBackupQuery::Elements &
bool compareRestoredTableDef(const IAST & restored_table_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context);
bool compareRestoredDatabaseDef(const IAST & restored_database_create_query, const IAST & create_query_from_backup, const ContextPtr & global_context);
/// Returns true if this table should be skipped while making a backup because it's an inner table.
bool isInnerTable(const QualifiedTableName & table_name);
bool isInnerTable(const String & database_name, const String & table_name);
}
}

View File

@ -564,7 +564,7 @@ void BackupsWorker::doBackup(
/// Checks access rights if this is not ON CLUSTER query.
/// (If this is ON CLUSTER query executeDDLQueryOnCluster() will check access rights later.)
auto required_access = getRequiredAccessToBackup(backup_query->elements);
auto required_access = BackupUtils::getRequiredAccessToBackup(backup_query->elements);
if (!on_cluster)
context->checkAccess(required_access);

View File

@ -124,7 +124,7 @@ void RestorerFromBackup::run(Mode mode)
restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num);
/// Do renaming in the create queries according to the renaming config.
renaming_map = makeRenamingMapFromBackupQuery(restore_query_elements);
renaming_map = BackupUtils::makeRenamingMap(restore_query_elements);
/// Calculate the root path in the backup for restoring, it's either empty or has the format "shards/<shard_num>/replicas/<replica_num>/".
findRootPathsInBackup();
@ -346,12 +346,12 @@ void RestorerFromBackup::findDatabasesAndTablesInBackup()
{
case ASTBackupQuery::ElementType::TABLE:
{
findTableInBackup({element.database_name, element.table_name}, element.partitions);
findTableInBackup({element.database_name, element.table_name}, /* skip_if_inner_table= */ false, element.partitions);
break;
}
case ASTBackupQuery::ElementType::TEMPORARY_TABLE:
{
findTableInBackup({DatabaseCatalog::TEMPORARY_DATABASE, element.table_name}, element.partitions);
findTableInBackup({DatabaseCatalog::TEMPORARY_DATABASE, element.table_name}, /* skip_if_inner_table= */ false, element.partitions);
break;
}
case ASTBackupQuery::ElementType::DATABASE:
@ -370,14 +370,14 @@ void RestorerFromBackup::findDatabasesAndTablesInBackup()
LOG_INFO(log, "Will restore {} databases and {} tables", getNumDatabases(), getNumTables());
}
void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions)
void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional<ASTs> & partitions)
{
schedule(
[this, table_name_in_backup, partitions]() { findTableInBackupImpl(table_name_in_backup, partitions); },
[this, table_name_in_backup, skip_if_inner_table, partitions]() { findTableInBackupImpl(table_name_in_backup, skip_if_inner_table, partitions); },
"Restore_FindTbl");
}
void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions)
void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional<ASTs> & partitions)
{
bool is_temporary_table = (table_name_in_backup.database == DatabaseCatalog::TEMPORARY_DATABASE);
@ -422,6 +422,10 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_
= *root_path_in_use / "data" / escapeForFileName(table_name_in_backup.database) / escapeForFileName(table_name_in_backup.table);
}
QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup);
if (skip_if_inner_table && BackupUtils::isInnerTable(table_name))
return;
auto read_buffer = backup->readFile(*metadata_path);
String create_query_str;
readStringUntilEOF(create_query_str, *read_buffer);
@ -432,8 +436,6 @@ void RestorerFromBackup::findTableInBackupImpl(const QualifiedTableName & table_
renameDatabaseAndTableNameInCreateQuery(create_table_query, renaming_map, context->getGlobalContext());
String create_table_query_str = serializeAST(*create_table_query);
QualifiedTableName table_name = renaming_map.getNewTableName(table_name_in_backup);
bool is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table});
auto table_dependencies = getDependenciesFromCreateQuery(context, table_name, create_table_query);
bool table_has_data = backup->hasFiles(data_path_in_backup);
@ -568,7 +570,7 @@ void RestorerFromBackup::findDatabaseInBackupImpl(const String & database_name_i
if (except_table_names.contains({database_name_in_backup, table_name_in_backup}))
continue;
findTableInBackup({database_name_in_backup, table_name_in_backup}, /* partitions= */ {});
findTableInBackup({database_name_in_backup, table_name_in_backup}, /* skip_if_inner_table= */ true, /* partitions= */ {});
}
}
@ -767,7 +769,7 @@ void RestorerFromBackup::checkDatabase(const String & database_name)
ASTPtr existing_database_def = database->getCreateDatabaseQuery();
ASTPtr database_def_from_backup = database_info.create_database_query;
if (!compareRestoredDatabaseDef(*existing_database_def, *database_def_from_backup, context->getGlobalContext()))
if (!BackupUtils::compareRestoredDatabaseDef(*existing_database_def, *database_def_from_backup, context->getGlobalContext()))
{
throw Exception(
ErrorCodes::CANNOT_RESTORE_DATABASE,
@ -938,7 +940,7 @@ void RestorerFromBackup::checkTable(const QualifiedTableName & table_name)
{
ASTPtr existing_table_def = database->getCreateTableQuery(resolved_id.table_name, context);
ASTPtr table_def_from_backup = table_info.create_table_query;
if (!compareRestoredTableDef(*existing_table_def, *table_def_from_backup, context->getGlobalContext()))
if (!BackupUtils::compareRestoredTableDef(*existing_table_def, *table_def_from_backup, context->getGlobalContext()))
{
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,

View File

@ -92,8 +92,8 @@ private:
void findRootPathsInBackup();
void findDatabasesAndTablesInBackup();
void findTableInBackup(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions);
void findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, const std::optional<ASTs> & partitions);
void findTableInBackup(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional<ASTs> & partitions);
void findTableInBackupImpl(const QualifiedTableName & table_name_in_backup, bool skip_if_inner_table, const std::optional<ASTs> & partitions);
void findDatabaseInBackup(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names);
void findDatabaseInBackupImpl(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names);
void findEverythingInBackup(const std::set<String> & except_database_names, const std::set<DatabaseAndTableName> & except_table_names);

View File

@ -329,12 +329,11 @@ void ClientBase::setupSignalHandler()
}
ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_multi_statements) const
ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements, bool is_interactive, bool ignore_error)
{
std::unique_ptr<IParserBase> parser;
ASTPtr res;
const auto & settings = global_context->getSettingsRef();
size_t max_length = 0;
if (!allow_multi_statements)
@ -343,11 +342,11 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu
const Dialect & dialect = settings.dialect;
if (dialect == Dialect::kusto)
parser = std::make_unique<ParserKQLStatement>(end, global_context->getSettings().allow_settings_after_format_in_insert);
parser = std::make_unique<ParserKQLStatement>(end, settings.allow_settings_after_format_in_insert);
else if (dialect == Dialect::prql)
parser = std::make_unique<ParserPRQLQuery>(max_length, settings.max_parser_depth, settings.max_parser_backtracks);
else
parser = std::make_unique<ParserQuery>(end, global_context->getSettings().allow_settings_after_format_in_insert);
parser = std::make_unique<ParserQuery>(end, settings.allow_settings_after_format_in_insert);
if (is_interactive || ignore_error)
{
@ -916,7 +915,11 @@ void ClientBase::processTextAsSingleQuery(const String & full_query)
/// Some parts of a query (result output and formatting) are executed
/// client-side. Thus we need to parse the query.
const char * begin = full_query.data();
auto parsed_query = parseQuery(begin, begin + full_query.size(), false);
auto parsed_query = parseQuery(begin, begin + full_query.size(),
global_context->getSettingsRef(),
/*allow_multi_statements=*/ false,
is_interactive,
ignore_error);
if (!parsed_query)
return;
@ -2089,7 +2092,11 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText(
this_query_end = this_query_begin;
try
{
parsed_query = parseQuery(this_query_end, all_queries_end, true);
parsed_query = parseQuery(this_query_end, all_queries_end,
global_context->getSettingsRef(),
/*allow_multi_statements=*/ true,
is_interactive,
ignore_error);
}
catch (const Exception & e)
{
@ -2470,9 +2477,9 @@ void ClientBase::runInteractive()
{
/// Load suggestion data from the server.
if (global_context->getApplicationType() == Context::ApplicationType::CLIENT)
suggest->load<Connection>(global_context, connection_parameters, config().getInt("suggestion_limit"));
suggest->load<Connection>(global_context, connection_parameters, config().getInt("suggestion_limit"), wait_for_suggestions_to_load);
else if (global_context->getApplicationType() == Context::ApplicationType::LOCAL)
suggest->load<LocalConnection>(global_context, connection_parameters, config().getInt("suggestion_limit"));
suggest->load<LocalConnection>(global_context, connection_parameters, config().getInt("suggestion_limit"), wait_for_suggestions_to_load);
}
if (home_path.empty())
@ -2968,6 +2975,7 @@ void ClientBase::init(int argc, char ** argv)
("progress", po::value<ProgressOption>()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off")
("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.")
("wait_for_suggestions_to_load", "Load suggestion data synchonously.")
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
("echo", "in batch mode, print query before execution")
@ -3097,6 +3105,8 @@ void ClientBase::init(int argc, char ** argv)
config().setBool("echo", true);
if (options.count("disable_suggestion"))
config().setBool("disable_suggestion", true);
if (options.count("wait_for_suggestions_to_load"))
config().setBool("wait_for_suggestions_to_load", true);
if (options.count("suggestion_limit"))
config().setInt("suggestion_limit", options["suggestion_limit"].as<int>());
if (options.count("highlight"))

View File

@ -73,6 +73,7 @@ public:
void init(int argc, char ** argv);
std::vector<String> getAllRegisteredNames() const override { return cmd_options; }
static ASTPtr parseQuery(const char *& pos, const char * end, const Settings & settings, bool allow_multi_statements, bool is_interactive, bool ignore_error);
protected:
void runInteractive();
@ -98,7 +99,6 @@ protected:
ASTPtr parsed_query, std::optional<bool> echo_query_ = {}, bool report_error = false);
static void adjustQueryEnd(const char *& this_query_end, const char * all_queries_end, uint32_t max_parser_depth, uint32_t max_parser_backtracks);
ASTPtr parseQuery(const char *& pos, const char * end, bool allow_multi_statements) const;
static void setupSignalHandler();
bool executeMultiQuery(const String & all_queries_text);
@ -209,6 +209,7 @@ protected:
std::optional<Suggest> suggest;
bool load_suggestions = false;
bool wait_for_suggestions_to_load = false;
std::vector<String> queries; /// Queries passed via '--query'
std::vector<String> queries_files; /// If not empty, queries will be read from these files

View File

@ -1,11 +1,18 @@
#include "LocalConnection.h"
#include <memory>
#include <Client/ClientBase.h>
#include <Core/Protocol.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/executeQuery.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
#include <Processors/Executors/PushingAsyncPipelineExecutor.h>
#include <Processors/Executors/PushingPipelineExecutor.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <QueryPipeline/QueryPipeline.h>
#include <QueryPipeline/Pipe.h>
#include <Parsers/ASTInsertQuery.h>
#include <Storages/IStorage.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/CurrentThread.h>
@ -22,12 +29,13 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
LocalConnection::LocalConnection(ContextPtr context_, bool send_progress_, bool send_profile_events_, const String & server_display_name_)
LocalConnection::LocalConnection(ContextPtr context_, ReadBuffer * in_, bool send_progress_, bool send_profile_events_, const String & server_display_name_)
: WithContext(context_)
, session(getContext(), ClientInfo::Interface::LOCAL)
, send_progress(send_progress_)
, send_profile_events(send_profile_events_)
, server_display_name(server_display_name_)
, in(in_)
{
/// Authenticate and create a context to execute queries.
session.authenticate("default", "", Poco::Net::SocketAddress{});
@ -130,6 +138,57 @@ void LocalConnection::sendQuery(
next_packet_type.reset();
/// Prepare input() function
query_context->setInputInitializer([this] (ContextPtr context, const StoragePtr & input_storage)
{
if (context != query_context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected context in Input initializer");
auto metadata_snapshot = input_storage->getInMemoryMetadataPtr();
Block sample = metadata_snapshot->getSampleBlock();
next_packet_type = Protocol::Server::Data;
state->block = sample;
String current_format = "Values";
const char * begin = state->query.data();
auto parsed_query = ClientBase::parseQuery(begin, begin + state->query.size(),
context->getSettingsRef(),
/*allow_multi_statements=*/ false,
/*is_interactive=*/ false,
/*ignore_error=*/ false);
if (const auto * insert = parsed_query->as<ASTInsertQuery>())
{
if (!insert->format.empty())
current_format = insert->format;
}
auto source = context->getInputFormat(current_format, *in, sample, context->getSettingsRef().max_insert_block_size);
Pipe pipe(source);
auto columns_description = metadata_snapshot->getColumns();
if (columns_description.hasDefaults())
{
pipe.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AddingDefaultsTransform>(header, columns_description, *source, context);
});
}
state->input_pipeline = std::make_unique<QueryPipeline>(std::move(pipe));
state->input_pipeline_executor = std::make_unique<PullingAsyncPipelineExecutor>(*state->input_pipeline);
});
query_context->setInputBlocksReaderCallback([this] (ContextPtr context) -> Block
{
if (context != query_context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected context in InputBlocksReader");
Block block;
state->input_pipeline_executor->pull(block);
return block;
});
try
{
state->io = executeQuery(state->query, query_context, QueryFlags{}, state->stage).second;
@ -537,11 +596,12 @@ void LocalConnection::sendMergeTreeReadTaskResponse(const ParallelReadResponse &
ServerConnectionPtr LocalConnection::createConnection(
const ConnectionParameters &,
ContextPtr current_context,
ReadBuffer * in,
bool send_progress,
bool send_profile_events,
const String & server_display_name)
{
return std::make_unique<LocalConnection>(current_context, send_progress, send_profile_events, server_display_name);
return std::make_unique<LocalConnection>(current_context, in, send_progress, send_profile_events, server_display_name);
}

View File

@ -15,6 +15,8 @@ namespace DB
class PullingAsyncPipelineExecutor;
class PushingAsyncPipelineExecutor;
class PushingPipelineExecutor;
class QueryPipeline;
class ReadBuffer;
/// State of query processing.
struct LocalQueryState
@ -31,6 +33,10 @@ struct LocalQueryState
std::unique_ptr<PullingAsyncPipelineExecutor> executor;
std::unique_ptr<PushingPipelineExecutor> pushing_executor;
std::unique_ptr<PushingAsyncPipelineExecutor> pushing_async_executor;
/// For sending data for input() function.
std::unique_ptr<QueryPipeline> input_pipeline;
std::unique_ptr<PullingAsyncPipelineExecutor> input_pipeline_executor;
InternalProfileEventsQueuePtr profile_queue;
std::unique_ptr<Exception> exception;
@ -64,7 +70,11 @@ class LocalConnection : public IServerConnection, WithContext
{
public:
explicit LocalConnection(
ContextPtr context_, bool send_progress_ = false, bool send_profile_events_ = false, const String & server_display_name_ = "");
ContextPtr context_,
ReadBuffer * in_,
bool send_progress_,
bool send_profile_events_,
const String & server_display_name_);
~LocalConnection() override;
@ -73,6 +83,7 @@ public:
static ServerConnectionPtr createConnection(
const ConnectionParameters & connection_parameters,
ContextPtr current_context,
ReadBuffer * in = nullptr,
bool send_progress = false,
bool send_profile_events = false,
const String & server_display_name = "");
@ -158,5 +169,7 @@ private:
String current_database;
ProfileEvents::ThreadIdToCountersSnapshot last_sent_snapshots;
ReadBuffer * in;
};
}

View File

@ -110,7 +110,7 @@ static String getLoadSuggestionQuery(Int32 suggestion_limit, bool basic_suggesti
}
template <typename ConnectionType>
void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit)
void Suggest::load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load)
{
loading_thread = std::thread([my_context = Context::createCopy(context), connection_parameters, suggestion_limit, this]
{
@ -152,6 +152,9 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p
/// Note that keyword suggestions are available even if we cannot load data from server.
});
if (wait_for_load)
loading_thread.join();
}
void Suggest::load(IServerConnection & connection,
@ -228,8 +231,8 @@ void Suggest::fillWordsFromBlock(const Block & block)
}
template
void Suggest::load<Connection>(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit);
void Suggest::load<Connection>(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load);
template
void Suggest::load<LocalConnection>(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit);
void Suggest::load<LocalConnection>(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load);
}

View File

@ -27,7 +27,7 @@ public:
/// Load suggestions for clickhouse-client.
template <typename ConnectionType>
void load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit);
void load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit, bool wait_for_load);
void load(IServerConnection & connection,
const ConnectionTimeouts & timeouts,

View File

@ -403,7 +403,7 @@ void ZooKeeperSetACLRequest::readImpl(ReadBuffer & in)
std::string ZooKeeperSetACLRequest::toStringImpl() const
{
return fmt::format("path = {}\n", "version = {}", path, version);
return fmt::format("path = {}\nversion = {}", path, version);
}
void ZooKeeperSetACLResponse::writeImpl(WriteBuffer & out) const
@ -457,7 +457,7 @@ void ZooKeeperCheckRequest::readImpl(ReadBuffer & in)
std::string ZooKeeperCheckRequest::toStringImpl() const
{
return fmt::format("path = {}\n", "version = {}", path, version);
return fmt::format("path = {}\nversion = {}", path, version);
}
void ZooKeeperErrorResponse::readImpl(ReadBuffer & in)

View File

@ -2,6 +2,7 @@
#include <string>
#include <cstdint>
#include <magic_enum.hpp>
namespace Coordination
@ -64,3 +65,12 @@ static constexpr int32_t DEFAULT_OPERATION_TIMEOUT_MS = 10000;
static constexpr int32_t DEFAULT_CONNECTION_TIMEOUT_MS = 1000;
}
/// This is used by fmt::format to print OpNum as strings.
/// All OpNum values should be in range [min, max] to be printed.
template <>
struct magic_enum::customize::enum_range<Coordination::OpNum>
{
static constexpr int min = -100;
static constexpr int max = 1000;
};

View File

@ -898,11 +898,13 @@ NearestFieldType<std::decay_t<T>> & Field::get()
template <typename T>
auto & Field::safeGet()
{
const Types::Which requested = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value;
const Types::Which target = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value;
if (which != requested)
/// We allow converting int64 <-> uint64, int64 <-> bool, uint64 <-> bool in safeGet().
if (target != which
&& (!isInt64OrUInt64orBoolFieldType(target) || !isInt64OrUInt64orBoolFieldType(which)))
throw Exception(ErrorCodes::BAD_GET,
"Bad get: has {}, requested {}", getTypeName(), requested);
"Bad get: has {}, requested {}", getTypeName(), target);
return get<T>();
}

View File

@ -89,7 +89,8 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"},
{"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"},
{"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"},
}},
{"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."},
}},
{"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"},
{"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"},
{"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"},
@ -129,7 +130,6 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."},
{"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."},
{"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."},
{"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."},
}},
{"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"},
{"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"},

View File

@ -2135,13 +2135,6 @@ ConjunctionNodes getConjunctionNodes(ActionsDAG::Node * predicate, std::unordere
}
}
// std::cerr << "Allowed " << conjunction.allowed.size() << std::endl;
// for (const auto & node : conjunction.allowed)
// std::cerr << node->result_name << std::endl;
// std::cerr << "Rejected " << conjunction.rejected.size() << std::endl;
// for (const auto & node : conjunction.rejected)
// std::cerr << node->result_name << std::endl;
return conjunction;
}
@ -2170,7 +2163,7 @@ ColumnsWithTypeAndName prepareFunctionArguments(const ActionsDAG::NodeRawConstPt
///
/// Result actions add single column with conjunction result (it is always first in outputs).
/// No other columns are added or removed.
ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs)
ActionsDAGPtr ActionsDAG::createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs)
{
if (conjunction.empty())
return nullptr;
@ -2265,9 +2258,9 @@ ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(NodeRawConstPtrs conjunctio
return actions;
}
ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
ActionsDAGPtr ActionsDAG::splitActionsForFilterPushDown(
const std::string & filter_name,
bool can_remove_filter,
bool removes_filter,
const Names & available_inputs,
const ColumnsWithTypeAndName & all_inputs)
{
@ -2321,16 +2314,232 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
}
}
auto actions = cloneActionsForConjunction(conjunction.allowed, all_inputs);
auto actions = createActionsForConjunction(conjunction.allowed, all_inputs);
if (!actions)
return nullptr;
/// Now, when actions are created, update the current DAG.
removeUnusedConjunctions(std::move(conjunction.rejected), predicate, removes_filter);
if (conjunction.rejected.empty())
return actions;
}
ActionsDAG::ActionsForJOINFilterPushDown ActionsDAG::splitActionsForJOINFilterPushDown(
const std::string & filter_name,
bool removes_filter,
const Names & left_stream_available_columns_to_push_down,
const Block & left_stream_header,
const Names & right_stream_available_columns_to_push_down,
const Block & right_stream_header,
const Names & equivalent_columns_to_push_down,
const std::unordered_map<std::string, ColumnWithTypeAndName> & equivalent_left_stream_column_to_right_stream_column,
const std::unordered_map<std::string, ColumnWithTypeAndName> & equivalent_right_stream_column_to_left_stream_column)
{
Node * predicate = const_cast<Node *>(tryFindInOutputs(filter_name));
if (!predicate)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Output nodes for ActionsDAG do not contain filter column name {}. DAG:\n{}",
filter_name,
dumpDAG());
/// If condition is constant let's do nothing.
/// It means there is nothing to push down or optimization was already applied.
if (predicate->type == ActionType::COLUMN)
return {};
auto get_input_nodes = [this](const Names & inputs_names)
{
std::unordered_set<const Node *> allowed_nodes;
std::unordered_map<std::string_view, std::list<const Node *>> inputs_map;
for (const auto & input_node : inputs)
inputs_map[input_node->result_name].emplace_back(input_node);
for (const auto & name : inputs_names)
{
auto & inputs_list = inputs_map[name];
if (inputs_list.empty())
continue;
allowed_nodes.emplace(inputs_list.front());
inputs_list.pop_front();
}
return allowed_nodes;
};
auto left_stream_allowed_nodes = get_input_nodes(left_stream_available_columns_to_push_down);
auto right_stream_allowed_nodes = get_input_nodes(right_stream_available_columns_to_push_down);
auto both_streams_allowed_nodes = get_input_nodes(equivalent_columns_to_push_down);
auto left_stream_push_down_conjunctions = getConjunctionNodes(predicate, left_stream_allowed_nodes);
auto right_stream_push_down_conjunctions = getConjunctionNodes(predicate, right_stream_allowed_nodes);
auto both_streams_push_down_conjunctions = getConjunctionNodes(predicate, both_streams_allowed_nodes);
NodeRawConstPtrs left_stream_allowed_conjunctions = std::move(left_stream_push_down_conjunctions.allowed);
NodeRawConstPtrs right_stream_allowed_conjunctions = std::move(right_stream_push_down_conjunctions.allowed);
std::unordered_set<const Node *> left_stream_allowed_conjunctions_set(left_stream_allowed_conjunctions.begin(), left_stream_allowed_conjunctions.end());
std::unordered_set<const Node *> right_stream_allowed_conjunctions_set(right_stream_allowed_conjunctions.begin(), right_stream_allowed_conjunctions.end());
for (const auto * both_streams_push_down_allowed_conjunction_node : both_streams_push_down_conjunctions.allowed)
{
if (!left_stream_allowed_conjunctions_set.contains(both_streams_push_down_allowed_conjunction_node))
left_stream_allowed_conjunctions.push_back(both_streams_push_down_allowed_conjunction_node);
if (!right_stream_allowed_conjunctions_set.contains(both_streams_push_down_allowed_conjunction_node))
right_stream_allowed_conjunctions.push_back(both_streams_push_down_allowed_conjunction_node);
}
std::unordered_set<const Node *> rejected_conjunctions_set;
rejected_conjunctions_set.insert(left_stream_push_down_conjunctions.rejected.begin(), left_stream_push_down_conjunctions.rejected.end());
rejected_conjunctions_set.insert(right_stream_push_down_conjunctions.rejected.begin(), right_stream_push_down_conjunctions.rejected.end());
rejected_conjunctions_set.insert(both_streams_push_down_conjunctions.rejected.begin(), both_streams_push_down_conjunctions.rejected.end());
for (const auto & left_stream_allowed_conjunction : left_stream_allowed_conjunctions)
rejected_conjunctions_set.erase(left_stream_allowed_conjunction);
for (const auto & right_stream_allowed_conjunction : right_stream_allowed_conjunctions)
rejected_conjunctions_set.erase(right_stream_allowed_conjunction);
NodeRawConstPtrs rejected_conjunctions(rejected_conjunctions_set.begin(), rejected_conjunctions_set.end());
if (rejected_conjunctions.size() == 1)
{
chassert(rejected_conjunctions.front()->result_type);
bool left_stream_push_constant = !left_stream_allowed_conjunctions.empty() && left_stream_allowed_conjunctions[0]->type == ActionType::COLUMN;
bool right_stream_push_constant = !right_stream_allowed_conjunctions.empty() && right_stream_allowed_conjunctions[0]->type == ActionType::COLUMN;
if ((left_stream_push_constant || right_stream_push_constant) && !rejected_conjunctions.front()->result_type->equals(*predicate->result_type))
{
/// No further optimization can be done
return {};
}
}
auto left_stream_filter_to_push_down = createActionsForConjunction(left_stream_allowed_conjunctions, left_stream_header.getColumnsWithTypeAndName());
auto right_stream_filter_to_push_down = createActionsForConjunction(right_stream_allowed_conjunctions, right_stream_header.getColumnsWithTypeAndName());
auto replace_equivalent_columns_in_filter = [](const ActionsDAGPtr & filter,
const Block & stream_header,
const std::unordered_map<std::string, ColumnWithTypeAndName> & columns_to_replace)
{
auto updated_filter = ActionsDAG::buildFilterActionsDAG({filter->getOutputs()[0]}, columns_to_replace);
chassert(updated_filter->getOutputs().size() == 1);
/** If result filter to left or right stream has column that is one of the stream inputs, we need distinguish filter column from
* actual input column. It is necessary because after filter step, filter column became constant column with value 1, and
* not all JOIN algorithms properly work with constants.
*
* Example: SELECT key FROM ( SELECT key FROM t1 ) AS t1 JOIN ( SELECT key FROM t1 ) AS t2 ON t1.key = t2.key WHERE key;
*/
const auto * stream_filter_node = updated_filter->getOutputs()[0];
if (stream_header.has(stream_filter_node->result_name))
{
const auto & alias_node = updated_filter->addAlias(*stream_filter_node, "__filter" + stream_filter_node->result_name);
updated_filter->getOutputs()[0] = &alias_node;
}
std::unordered_map<std::string, std::list<const Node *>> updated_filter_inputs;
for (const auto & input : updated_filter->getInputs())
updated_filter_inputs[input->result_name].push_back(input);
for (const auto & input : filter->getInputs())
{
if (updated_filter_inputs.contains(input->result_name))
continue;
const Node * updated_filter_input_node = nullptr;
auto it = columns_to_replace.find(input->result_name);
if (it != columns_to_replace.end())
updated_filter_input_node = &updated_filter->addInput(it->second);
else
updated_filter_input_node = &updated_filter->addInput({input->column, input->result_type, input->result_name});
updated_filter_inputs[input->result_name].push_back(updated_filter_input_node);
}
for (const auto & input_column : stream_header.getColumnsWithTypeAndName())
{
const Node * input;
auto & list = updated_filter_inputs[input_column.name];
if (list.empty())
{
input = &updated_filter->addInput(input_column);
}
else
{
input = list.front();
list.pop_front();
}
if (input != updated_filter->getOutputs()[0])
updated_filter->outputs.push_back(input);
}
return updated_filter;
};
if (left_stream_filter_to_push_down)
left_stream_filter_to_push_down = replace_equivalent_columns_in_filter(left_stream_filter_to_push_down,
left_stream_header,
equivalent_right_stream_column_to_left_stream_column);
if (right_stream_filter_to_push_down)
right_stream_filter_to_push_down = replace_equivalent_columns_in_filter(right_stream_filter_to_push_down,
right_stream_header,
equivalent_left_stream_column_to_right_stream_column);
/*
* We should check the presence of a split filter column name in stream columns to avoid removing the required column.
*
* Example:
* A filter expression is `a AND b = c`, but `b` and `c` belong to another side of the join and not in allowed columns to push down,
* so the final split filter is just `a`.
* In this case `a` can be in stream columns but not `and(a, equals(b, c))`.
*/
bool left_stream_filter_removes_filter = true;
bool right_stream_filter_removes_filter = true;
if (left_stream_filter_to_push_down)
{
const auto & left_stream_filter_column_name = left_stream_filter_to_push_down->getOutputs()[0]->result_name;
left_stream_filter_removes_filter = !left_stream_header.has(left_stream_filter_column_name);
}
if (right_stream_filter_to_push_down)
{
const auto & right_stream_filter_column_name = right_stream_filter_to_push_down->getOutputs()[0]->result_name;
right_stream_filter_removes_filter = !right_stream_header.has(right_stream_filter_column_name);
}
ActionsDAG::ActionsForJOINFilterPushDown result
{
.left_stream_filter_to_push_down = std::move(left_stream_filter_to_push_down),
.left_stream_filter_removes_filter = left_stream_filter_removes_filter,
.right_stream_filter_to_push_down = std::move(right_stream_filter_to_push_down),
.right_stream_filter_removes_filter = right_stream_filter_removes_filter
};
if (!result.left_stream_filter_to_push_down && !result.right_stream_filter_to_push_down)
return result;
/// Now, when actions are created, update the current DAG.
removeUnusedConjunctions(std::move(rejected_conjunctions), predicate, removes_filter);
return result;
}
void ActionsDAG::removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter)
{
if (rejected_conjunctions.empty())
{
/// The whole predicate was split.
if (can_remove_filter)
if (removes_filter)
{
/// If filter column is not needed, remove it from output nodes.
std::erase_if(outputs, [&](const Node * node) { return node == predicate; });
@ -2362,7 +2571,7 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
{
/// Predicate is conjunction, where both allowed and rejected sets are not empty.
NodeRawConstPtrs new_children = std::move(conjunction.rejected);
NodeRawConstPtrs new_children = std::move(rejected_conjunctions);
if (new_children.size() == 1 && new_children.front()->result_type->equals(*predicate->result_type))
{
@ -2403,13 +2612,12 @@ ActionsDAGPtr ActionsDAG::cloneActionsForFilterPushDown(
std::unordered_set<const Node *> used_inputs;
for (const auto * input : inputs)
{
if (can_remove_filter && input == predicate)
if (removes_filter && input == predicate)
continue;
used_inputs.insert(input);
}
removeUnusedActions(used_inputs);
return actions;
}
static bool isColumnSortingPreserved(const ActionsDAG::Node * start_node, const String & sorted_column)
@ -2557,8 +2765,11 @@ ActionsDAGPtr ActionsDAG::buildFilterActionsDAG(
auto input_node_it = node_name_to_input_node_column.find(node->result_name);
if (input_node_it != node_name_to_input_node_column.end())
{
result_node = &result_dag->addInput(input_node_it->second);
node_to_result_node.emplace(node, result_node);
auto & result_input = result_inputs[input_node_it->second.name];
if (!result_input)
result_input = &result_dag->addInput(input_node_it->second);
node_to_result_node.emplace(node, result_input);
nodes_to_process.pop_back();
continue;
}

View File

@ -372,12 +372,46 @@ public:
/// columns will be transformed like `x, y, z` -> `z > 0, z, x, y` -(remove filter)-> `z, x, y`.
/// To avoid it, add inputs from `all_inputs` list,
/// so actions `x, y, z -> z > 0, x, y, z` -(remove filter)-> `x, y, z` will not change columns order.
ActionsDAGPtr cloneActionsForFilterPushDown(
ActionsDAGPtr splitActionsForFilterPushDown(
const std::string & filter_name,
bool can_remove_filter,
bool removes_filter,
const Names & available_inputs,
const ColumnsWithTypeAndName & all_inputs);
struct ActionsForJOINFilterPushDown
{
ActionsDAGPtr left_stream_filter_to_push_down;
bool left_stream_filter_removes_filter;
ActionsDAGPtr right_stream_filter_to_push_down;
bool right_stream_filter_removes_filter;
};
/** Split actions for JOIN filter push down.
*
* @param filter_name - name of filter node in current DAG.
* @param removes_filter - if filter is removed after it is applied.
* @param left_stream_available_columns_to_push_down - columns from left stream that are safe to use in push down conditions
* to left stream.
* @param left_stream_header - left stream header.
* @param right_stream_available_columns_to_push_down - columns from right stream that are safe to use in push down conditions
* to right stream.
* @param right_stream_header - right stream header.
* @param equivalent_columns_to_push_down - columns from left and right streams that are safe to use in push down conditions
* to left and right streams.
* @param equivalent_left_stream_column_to_right_stream_column - equivalent left stream column name to right stream column map.
* @param equivalent_right_stream_column_to_left_stream_column - equivalent right stream column name to left stream column map.
*/
ActionsForJOINFilterPushDown splitActionsForJOINFilterPushDown(
const std::string & filter_name,
bool removes_filter,
const Names & left_stream_available_columns_to_push_down,
const Block & left_stream_header,
const Names & right_stream_available_columns_to_push_down,
const Block & right_stream_header,
const Names & equivalent_columns_to_push_down,
const std::unordered_map<std::string, ColumnWithTypeAndName> & equivalent_left_stream_column_to_right_stream_column,
const std::unordered_map<std::string, ColumnWithTypeAndName> & equivalent_right_stream_column_to_left_stream_column);
bool
isSortingPreserved(const Block & input_header, const SortDescription & sort_description, const String & ignore_output_column = "") const;
@ -429,7 +463,9 @@ private:
void compileFunctions(size_t min_count_to_compile_expression, const std::unordered_set<const Node *> & lazy_executed_nodes = {});
#endif
static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
static ActionsDAGPtr createActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
void removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions, Node * predicate, bool removes_filter);
};
class FindOriginalNodeForOutputName

View File

@ -24,6 +24,7 @@ ColumnsDescription BackupLogElement::getColumnsDescription()
{
{"hostname", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "Hostname of the server executing the query."},
{"event_date", std::make_shared<DataTypeDate>(), "Date of the entry."},
{"event_time", std::make_shared<DataTypeDateTime>(), "Time of the entry."},
{"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6), "Time of the entry with microseconds precision."},
{"id", std::make_shared<DataTypeString>(), "Identifier of the backup or restore operation."},
{"name", std::make_shared<DataTypeString>(), "Name of the backup storage (the contents of the FROM or TO clause)."},
@ -48,6 +49,7 @@ void BackupLogElement::appendToBlock(MutableColumns & columns) const
size_t i = 0;
columns[i++]->insert(getFQDNOrHostName());
columns[i++]->insert(DateLUT::instance().toDayNum(std::chrono::system_clock::to_time_t(event_time)).toUnderType());
columns[i++]->insert(std::chrono::system_clock::to_time_t(event_time));
columns[i++]->insert(event_time_usec);
columns[i++]->insert(info.id);
columns[i++]->insert(info.name);

View File

@ -210,12 +210,8 @@ bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & p
if (!s_eq.ignore(pos, expected))
return false;
if (ParserKeyword(Keyword::TRUE_KEYWORD).ignore(pos, expected))
value = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(1)));
else if (ParserKeyword(Keyword::FALSE_KEYWORD).ignore(pos, expected))
value = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(0)));
/// for SETTINGS disk=disk(type='s3', path='', ...)
else if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk")
if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk")
{
tryGetIdentifierNameInto(name, change.name);
change.value = createFieldFromAST(function_ast);
@ -276,11 +272,7 @@ bool ParserSetQuery::parseNameValuePairWithParameterOrDefault(
}
/// Setting
if (ParserKeyword(Keyword::TRUE_KEYWORD).ignore(pos, expected))
node = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(1)));
else if (ParserKeyword(Keyword::FALSE_KEYWORD).ignore(pos, expected))
node = std::make_shared<ASTLiteral>(Field(static_cast<UInt64>(0)));
else if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk")
if (function_p.parse(pos, function_ast, expected) && function_ast->as<ASTFunction>()->name == "disk")
{
change.name = name;
change.value = createFieldFromAST(function_ast);

View File

@ -39,6 +39,7 @@ namespace ErrorCodes
extern const int UNSUPPORTED_METHOD;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int INCORRECT_QUERY;
}
namespace
@ -500,7 +501,41 @@ public:
ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node);
private:
using NodeNameAndNodeMinLevel = std::pair<std::string, size_t>;
class Levels
{
public:
explicit Levels(size_t level) { set(level); }
void set(size_t level)
{
check(level);
if (level)
mask |= (uint64_t(1) << (level - 1));
}
void reset(size_t level)
{
check(level);
if (level)
mask &= ~(uint64_t(1) << (level - 1));
}
void add(Levels levels) { mask |= levels.mask; }
size_t max() const { return 64 - getLeadingZeroBits(mask); }
private:
uint64_t mask = 0;
void check(size_t level)
{
if (level > 64)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Maximum lambda depth exceeded. Maximum 64.");
}
};
using NodeNameAndNodeMinLevel = std::pair<std::string, Levels>;
NodeNameAndNodeMinLevel visitImpl(QueryTreeNodePtr node);
@ -586,11 +621,11 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
column_source->getNodeType() == QueryTreeNodeType::LAMBDA &&
actions_stack[i].getScopeNode().get() == column_source.get())
{
return {column_node_name, i};
return {column_node_name, Levels(i)};
}
}
return {column_node_name, 0};
return {column_node_name, Levels(0)};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitConstant(const QueryTreeNodePtr & node)
@ -660,7 +695,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column);
}
return {constant_node_name, 0};
return {constant_node_name, Levels(0)};
}
@ -688,7 +723,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
auto lambda_actions_dag = std::make_shared<ActionsDAG>();
actions_stack.emplace_back(lambda_actions_dag, node);
auto [lambda_expression_node_name, level] = visitImpl(lambda_node.getExpression());
auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression());
lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name));
lambda_actions_dag->removeUnusedActions(Names(1, lambda_expression_node_name));
@ -699,8 +734,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
ActionsDAG::NodeRawConstPtrs lambda_children;
Names required_column_names = lambda_actions->getRequiredColumns();
if (level == actions_stack.size() - 1)
--level;
actions_stack.pop_back();
levels.reset(actions_stack.size());
size_t level = levels.max();
const auto & lambda_argument_names = lambda_node.getArgumentNames();
@ -718,7 +754,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
auto lambda_node_name = calculateActionNodeName(node, *planner_context);
auto function_capture = std::make_shared<FunctionCaptureOverloadResolver>(
lambda_actions, captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name);
actions_stack.pop_back();
// TODO: Pass IFunctionBase here not FunctionCaptureOverloadResolver.
const auto * actions_node = actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), function_capture);
@ -735,7 +770,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
actions_stack_node.addInputColumnIfNecessary(lambda_node_name, result_type);
}
return {lambda_node_name, level};
return {lambda_node_name, levels};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node)
@ -799,7 +834,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma
actions_stack_node.addInputConstantColumnIfNecessary(column.name, column);
}
return {column.name, 0};
return {column.name, Levels(0)};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node)
@ -833,7 +868,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
size_t index_hint_function_level = actions_stack.size() - 1;
actions_stack[index_hint_function_level].addFunctionIfNecessary(function_node_name, {}, index_hint_function_overload_resolver);
return {function_node_name, index_hint_function_level};
return {function_node_name, Levels(index_hint_function_level)};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitFunction(const QueryTreeNodePtr & node)
@ -868,7 +903,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType());
}
return {function_node_name, 0};
return {function_node_name, Levels(0)};
}
const auto & function_arguments = function_node.getArguments().getNodes();
@ -877,14 +912,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
Names function_arguments_node_names;
function_arguments_node_names.reserve(function_arguments_size);
size_t level = 0;
Levels levels(0);
for (size_t function_argument_index = 0; function_argument_index < function_arguments_size; ++function_argument_index)
{
if (in_function_second_argument_node_name_with_level && function_argument_index == 1)
{
auto & [node_name, node_min_level] = *in_function_second_argument_node_name_with_level;
auto & [node_name, node_levels] = *in_function_second_argument_node_name_with_level;
function_arguments_node_names.push_back(std::move(node_name));
level = std::max(level, node_min_level);
levels.add(node_levels);
continue;
}
@ -892,20 +927,21 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
if (argument->getNodeType() == QueryTreeNodeType::LAMBDA)
{
auto [node_name, node_min_level] = visitLambda(argument);
auto [node_name, node_levels] = visitLambda(argument);
function_arguments_node_names.push_back(std::move(node_name));
level = std::max(level, node_min_level);
levels.add(node_levels);
continue;
}
auto [node_name, node_min_level] = visitImpl(argument);
auto [node_name, node_levels] = visitImpl(argument);
function_arguments_node_names.push_back(std::move(node_name));
level = std::max(level, node_min_level);
levels.add(node_levels);
}
ActionsDAG::NodeRawConstPtrs children;
children.reserve(function_arguments_size);
size_t level = levels.max();
for (auto & function_argument_node_name : function_arguments_node_names)
children.push_back(actions_stack[level].getNodeOrThrow(function_argument_node_name));
@ -930,7 +966,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
actions_stack_node.addInputColumnIfNecessary(function_node_name, function_node.getResultType());
}
return {function_node_name, level};
return {function_node_name, levels};
}
}

View File

@ -24,6 +24,7 @@ public:
void describeActions(FormatSettings & settings) const override;
const ActionsDAGPtr & getExpression() const { return actions_dag; }
ActionsDAGPtr & getExpression() { return actions_dag; }
const String & getFilterColumnName() const { return filter_column_name; }
bool removesFilterColumn() const { return remove_filter_column; }

View File

@ -100,7 +100,7 @@ static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node)
return res;
}
static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & allowed_inputs, size_t child_idx = 0)
static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & available_inputs, size_t child_idx = 0)
{
QueryPlan::Node * child_node = parent_node->children.front();
checkChildrenSize(child_node, child_idx + 1);
@ -114,14 +114,12 @@ static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & al
bool removes_filter = filter->removesFilterColumn();
const auto & all_inputs = child->getInputStreams()[child_idx].header.getColumnsWithTypeAndName();
auto split_filter = expression->cloneActionsForFilterPushDown(filter_column_name, removes_filter, allowed_inputs, all_inputs);
return split_filter;
return expression->splitActionsForFilterPushDown(filter_column_name, removes_filter, available_inputs, all_inputs);
}
static size_t
tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter,
bool can_remove_filter = true, size_t child_idx = 0)
addNewFilterStepOrThrow(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const ActionsDAGPtr & split_filter,
bool can_remove_filter = true, size_t child_idx = 0, bool update_parent_filter = true)
{
QueryPlan::Node * child_node = parent_node->children.front();
checkChildrenSize(child_node, child_idx + 1);
@ -134,21 +132,18 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con
const auto & filter_column_name = filter->getFilterColumnName();
const auto * filter_node = expression->tryFindInOutputs(filter_column_name);
if (!filter_node && !filter->removesFilterColumn())
if (update_parent_filter && !filter_node && !filter->removesFilterColumn())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}",
filter_column_name, expression->dumpDAG());
/// Filter column was replaced to constant.
const bool filter_is_constant = filter_node && filter_node->column && isColumnConst(*filter_node->column);
/// Add new Filter step before Aggregating.
/// Expression/Filter -> Aggregating -> Something
/// Add new Filter step before Child.
/// Expression/Filter -> Child -> Something
auto & node = nodes.emplace_back();
node.children.emplace_back(&node);
std::swap(node.children[0], child_node->children[child_idx]);
/// Expression/Filter -> Aggregating -> Filter -> Something
/// Expression/Filter -> Child -> Filter -> Something
/// New filter column is the first one.
String split_filter_column_name = split_filter->getOutputs().front()->result_name;
@ -171,12 +166,22 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con
ErrorCodes::LOGICAL_ERROR, "We are trying to push down a filter through a step for which we cannot update input stream");
}
if (!filter_node || filter_is_constant)
/// This means that all predicates of filter were pushed down.
/// Replace current actions to expression, as we don't need to filter anything.
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), expression);
else
filter->updateInputStream(child->getOutputStream());
if (update_parent_filter)
{
/// Filter column was replaced to constant.
const bool filter_is_constant = filter_node && filter_node->column && isColumnConst(*filter_node->column);
if (!filter_node || filter_is_constant)
{
/// This means that all predicates of filter were pushed down.
/// Replace current actions to expression, as we don't need to filter anything.
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), expression);
}
else
{
filter->updateInputStream(child->getOutputStream());
}
}
return 3;
}
@ -186,7 +191,7 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con
bool can_remove_filter = true, size_t child_idx = 0)
{
if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx))
return tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx);
return addNewFilterStepOrThrow(parent_node, nodes, split_filter, can_remove_filter, child_idx);
return 0;
}
@ -204,6 +209,204 @@ static size_t simplePushDownOverStep(QueryPlan::Node * parent_node, QueryPlan::N
return 0;
}
static size_t tryPushDownOverJoinStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, QueryPlanStepPtr & child)
{
auto & parent = parent_node->step;
auto * filter = assert_cast<FilterStep *>(parent.get());
auto * join = typeid_cast<JoinStep *>(child.get());
auto * filled_join = typeid_cast<FilledJoinStep *>(child.get());
if (!join && !filled_join)
return 0;
/** For equivalent JOIN with condition `ON lhs.x_1 = rhs.y_1 AND lhs.x_2 = rhs.y_2 ...`, we can build equivalent sets of columns and this
* will allow to push conditions that only use columns from equivalent sets to both sides of JOIN, without considering JOIN type.
*
* For example: `FROM lhs INNER JOIN rhs ON lhs.id = rhs.id AND lhs.value = rhs.value`
* In this example columns `id` and `value` from both tables are equivalent.
*
* During filter push down for different JOIN types filter push down logic is different:
*
* 1. For INNER JOIN we can push all valid conditions to both sides of JOIN. We also can push all valid conditions that use columns from
* equivalent sets to both sides of JOIN.
* 2. For LEFT/RIGHT JOIN we can push conditions that use columns from LEFT/RIGHT stream to LEFT/RIGHT JOIN side. We can also push conditions
* that use columns from LEFT/RIGHT equivalent sets to RIGHT/LEFT JOIN side.
*
* Additional filter push down optimizations:
* 1. TODO: Support building equivalent sets for more than 2 JOINS. It is possible, but will require more complex analysis step.
* 2. TODO: Support building equivalent sets for JOINs with more than 1 clause.
* 3. TODO: For LEFT/RIGHT JOIN, we can assume that RIGHT/LEFT columns used in filter will be default/NULL constants and
* check if filter will always be false, in those scenario we can transform LEFT/RIGHT JOIN into INNER JOIN and push conditions to both tables.
* 4. TODO: It is possible to pull up filter conditions from LEFT/RIGHT stream and push conditions that use columns from LEFT/RIGHT equivalent sets
* to RIGHT/LEFT JOIN side.
*/
const auto & join_header = child->getOutputStream().header;
const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin();
const auto & left_stream_input_header = child->getInputStreams().front().header;
const auto & right_stream_input_header = child->getInputStreams().back().header;
if (table_join.kind() == JoinKind::Full)
return 0;
std::unordered_map<std::string, ColumnWithTypeAndName> equivalent_left_stream_column_to_right_stream_column;
std::unordered_map<std::string, ColumnWithTypeAndName> equivalent_right_stream_column_to_left_stream_column;
bool has_single_clause = table_join.getClauses().size() == 1;
if (has_single_clause)
{
const auto & join_clause = table_join.getClauses()[0];
size_t key_names_size = join_clause.key_names_left.size();
for (size_t i = 0; i < key_names_size; ++i)
{
const auto & left_table_key_name = join_clause.key_names_left[i];
const auto & right_table_key_name = join_clause.key_names_right[i];
if (!join_header.has(left_table_key_name) || !join_header.has(right_table_key_name))
continue;
const auto & left_table_column = left_stream_input_header.getByName(left_table_key_name);
const auto & right_table_column = right_stream_input_header.getByName(right_table_key_name);
if (!left_table_column.type->equals(*right_table_column.type))
continue;
equivalent_left_stream_column_to_right_stream_column[left_table_key_name] = right_table_column;
equivalent_right_stream_column_to_left_stream_column[right_table_key_name] = left_table_column;
}
}
auto get_available_columns_for_filter = [&](bool push_to_left_stream, bool filter_push_down_input_columns_available)
{
Names available_input_columns_for_filter;
if (!filter_push_down_input_columns_available)
return available_input_columns_for_filter;
const auto & input_header = push_to_left_stream ? left_stream_input_header : right_stream_input_header;
const auto & input_columns_names = input_header.getNames();
for (const auto & name : input_columns_names)
{
if (!join_header.has(name))
continue;
/// Skip if type is changed. Push down expression expect equal types.
if (!input_header.getByName(name).type->equals(*join_header.getByName(name).type))
continue;
available_input_columns_for_filter.push_back(name);
}
return available_input_columns_for_filter;
};
bool left_stream_filter_push_down_input_columns_available = true;
bool right_stream_filter_push_down_input_columns_available = true;
if (table_join.kind() == JoinKind::Left)
right_stream_filter_push_down_input_columns_available = false;
else if (table_join.kind() == JoinKind::Right)
left_stream_filter_push_down_input_columns_available = false;
/** We disable push down to right table in cases:
* 1. Right side is already filled. Example: JOIN with Dictionary.
* 2. ASOF Right join is not supported.
*/
bool allow_push_down_to_right = join && join->allowPushDownToRight() && table_join.strictness() != JoinStrictness::Asof;
if (!allow_push_down_to_right)
right_stream_filter_push_down_input_columns_available = false;
Names equivalent_columns_to_push_down;
if (left_stream_filter_push_down_input_columns_available)
{
for (const auto & [name, _] : equivalent_left_stream_column_to_right_stream_column)
equivalent_columns_to_push_down.push_back(name);
}
if (right_stream_filter_push_down_input_columns_available)
{
for (const auto & [name, _] : equivalent_right_stream_column_to_left_stream_column)
equivalent_columns_to_push_down.push_back(name);
}
Names left_stream_available_columns_to_push_down = get_available_columns_for_filter(true /*push_to_left_stream*/, left_stream_filter_push_down_input_columns_available);
Names right_stream_available_columns_to_push_down = get_available_columns_for_filter(false /*push_to_left_stream*/, right_stream_filter_push_down_input_columns_available);
auto join_filter_push_down_actions = filter->getExpression()->splitActionsForJOINFilterPushDown(filter->getFilterColumnName(),
filter->removesFilterColumn(),
left_stream_available_columns_to_push_down,
left_stream_input_header.getColumnsWithTypeAndName(),
right_stream_available_columns_to_push_down,
right_stream_input_header.getColumnsWithTypeAndName(),
equivalent_columns_to_push_down,
equivalent_left_stream_column_to_right_stream_column,
equivalent_right_stream_column_to_left_stream_column);
size_t updated_steps = 0;
if (join_filter_push_down_actions.left_stream_filter_to_push_down)
{
updated_steps += addNewFilterStepOrThrow(parent_node,
nodes,
join_filter_push_down_actions.left_stream_filter_to_push_down,
join_filter_push_down_actions.left_stream_filter_removes_filter,
0 /*child_idx*/,
false /*update_parent_filter*/);
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
"Pushed down filter {} to the {} side of join",
join_filter_push_down_actions.left_stream_filter_to_push_down->getOutputs()[0]->result_name,
JoinKind::Left);
}
if (join_filter_push_down_actions.right_stream_filter_to_push_down)
{
updated_steps += addNewFilterStepOrThrow(parent_node,
nodes,
join_filter_push_down_actions.right_stream_filter_to_push_down,
join_filter_push_down_actions.right_stream_filter_removes_filter,
1 /*child_idx*/,
false /*update_parent_filter*/);
LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"),
"Pushed down filter {} to the {} side of join",
join_filter_push_down_actions.right_stream_filter_to_push_down->getOutputs()[0]->result_name,
JoinKind::Right);
}
if (updated_steps > 0)
{
const auto & filter_column_name = filter->getFilterColumnName();
const auto & filter_expression = filter->getExpression();
const auto * filter_node = filter_expression->tryFindInOutputs(filter_column_name);
if (!filter_node && !filter->removesFilterColumn())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}",
filter_column_name, filter_expression->dumpDAG());
/// Filter column was replaced to constant.
const bool filter_is_constant = filter_node && filter_node->column && isColumnConst(*filter_node->column);
if (!filter_node || filter_is_constant)
{
/// This means that all predicates of filter were pushed down.
/// Replace current actions to expression, as we don't need to filter anything.
parent = std::make_unique<ExpressionStep>(child->getOutputStream(), filter_expression);
}
else
{
filter->updateInputStream(child->getOutputStream());
}
}
return updated_steps;
}
size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes)
{
if (parent_node->children.size() != 1)
@ -317,9 +520,6 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
if (!keys.contains(column.name))
allowed_inputs.push_back(column.name);
// for (const auto & name : allowed_inputs)
// std::cerr << name << std::endl;
if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs))
return updated_steps;
}
@ -327,77 +527,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
if (auto updated_steps = simplePushDownOverStep<DistinctStep>(parent_node, nodes, child))
return updated_steps;
auto * join = typeid_cast<JoinStep *>(child.get());
auto * filled_join = typeid_cast<FilledJoinStep *>(child.get());
if (join || filled_join)
{
auto join_push_down = [&](JoinKind kind) -> size_t
{
const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin();
/// Only inner, cross and left(/right) 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.
if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind)
return 0;
/// There is no ASOF Right join, so we're talking about pushing to the right side
if (kind == JoinKind::Right && table_join.strictness() == JoinStrictness::Asof)
return 0;
bool is_left = kind == JoinKind::Left;
const auto & input_header = is_left ? child->getInputStreams().front().header : child->getInputStreams().back().header;
const auto & res_header = child->getOutputStream().header;
Names allowed_keys;
const auto & source_columns = input_header.getNames();
for (const auto & name : source_columns)
{
/// Skip key if it is renamed.
/// I don't know if it is possible. Just in case.
if (!input_header.has(name) || !res_header.has(name))
continue;
/// Skip if type is changed. Push down expression expect equal types.
if (!input_header.getByName(name).type->equals(*res_header.getByName(name).type))
continue;
allowed_keys.push_back(name);
}
/// For left JOIN, push down to the first child; for right - to the second one.
const auto child_idx = is_left ? 0 : 1;
ActionsDAGPtr split_filter = splitFilter(parent_node, allowed_keys, child_idx);
if (!split_filter)
return 0;
/*
* We should check the presence of a split filter column name in `source_columns` to avoid removing the required column.
*
* Example:
* A filter expression is `a AND b = c`, but `b` and `c` belong to another side of the join and not in `allowed_keys`, so the final split filter is just `a`.
* In this case `a` can be in `source_columns` but not `and(a, equals(b, c))`.
*
* New filter column is the first one.
*/
const String & split_filter_column_name = split_filter->getOutputs().front()->result_name;
bool can_remove_filter = source_columns.end() == std::find(source_columns.begin(), source_columns.end(), split_filter_column_name);
const size_t updated_steps = tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx);
if (updated_steps > 0)
{
LOG_DEBUG(getLogger("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind);
}
return updated_steps;
};
if (size_t updated_steps = join_push_down(JoinKind::Left))
return updated_steps;
/// For full sorting merge join we push down both to the left and right tables, because left and right streams are not independent.
if (join && join->allowPushDownToRight())
{
if (size_t updated_steps = join_push_down(JoinKind::Right))
return updated_steps;
}
}
if (auto updated_steps = tryPushDownOverJoinStep(parent_node, nodes, child))
return updated_steps;
/// TODO.
/// We can filter earlier if expression does not depend on WITH FILL columns.

View File

@ -7074,7 +7074,7 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(
return checkStructureAndGetMergeTreeData(*source_table, src_snapshot, my_snapshot);
}
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAndLoadDataPart(
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAndLoadDataPartOnSameDisk(
const MergeTreeData::DataPartPtr & src_part,
const String & tmp_part_prefix,
const MergeTreePartInfo & dst_part_info,
@ -7084,23 +7084,28 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
const WriteSettings & write_settings)
{
chassert(!isStaticStorage());
bool on_same_disk = false;
for (const DiskPtr & disk : this->getStoragePolicy()->getDisks())
/// Check that the storage policy contains the disk where the src_part is located.
bool does_storage_policy_allow_same_disk = false;
for (const DiskPtr & disk : getStoragePolicy()->getDisks())
{
if (disk->getName() == src_part->getDataPartStorage().getDiskName())
{
on_same_disk = true;
does_storage_policy_allow_same_disk = true;
break;
}
}
if (!does_storage_policy_allow_same_disk)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Could not clone and load part {} because disk does not belong to storage policy",
quoteString(src_part->getDataPartStorage().getFullPath()));
String dst_part_name = src_part->getNewName(dst_part_info);
String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name);
/// Why it is needed if we only hardlink files?
/// Answer: In issue #59377, add copy when attach from different disk.
auto reservation = src_part->getDataPartStorage().reserve(src_part->getBytesOnDisk());
auto src_part_storage = src_part->getDataPartStoragePtr();
@ -7108,30 +7113,16 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
MergeTreeData::MutableDataPartPtr src_flushed_tmp_part;
String with_copy;
if (params.copy_instead_of_hardlink || !on_same_disk)
if (params.copy_instead_of_hardlink)
with_copy = " (copying data)";
std::shared_ptr<IDataPartStorage> dst_part_storage{};
if (on_same_disk && !params.copy_instead_of_hardlink)
{
dst_part_storage = src_part_storage->freeze(
relative_data_path,
tmp_dst_part_name,
read_settings,
write_settings,
/* save_metadata_callback= */ {},
params);
}
else
{
auto reservation_on_dst = getStoragePolicy()->reserve(src_part->getBytesOnDisk());
if (!reservation_on_dst)
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on disk.");
dst_part_storage = src_part_storage->clonePart(
this->getRelativeDataPath(), tmp_dst_part_name, reservation_on_dst->getDisk(), read_settings, write_settings, {}, {});
}
auto dst_part_storage = src_part_storage->freeze(
relative_data_path,
tmp_dst_part_name,
read_settings,
write_settings,
/* save_metadata_callback= */ {},
params);
if (params.metadata_version_to_write.has_value())
{
@ -7153,7 +7144,7 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
.withPartFormatFromDisk()
.build();
if (on_same_disk && !params.copy_instead_of_hardlink && params.hardlinked_files)
if (!params.copy_instead_of_hardlink && params.hardlinked_files)
{
params.hardlinked_files->source_part_name = src_part->name;
params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID();
@ -7197,7 +7188,6 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
return std::make_pair(dst_data_part, std::move(temporary_directory_lock));
}
String MergeTreeData::getFullPathOnDisk(const DiskPtr & disk) const
{
return disk->getPath() + relative_data_path;

View File

@ -839,7 +839,7 @@ public:
MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> cloneAndLoadDataPart(
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> cloneAndLoadDataPartOnSameDisk(
const MergeTreeData::DataPartPtr & src_part,
const String & tmp_part_prefix,
const MergeTreePartInfo & dst_part_info,

View File

@ -2146,7 +2146,7 @@ bool MutateTask::prepare()
scope_guard lock;
{
std::tie(part, lock) = ctx->data->cloneAndLoadDataPart(
std::tie(part, lock) = ctx->data->cloneAndLoadDataPartOnSameDisk(
ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, clone_params, ctx->context->getReadSettings(), ctx->context->getWriteSettings());
part->getDataPartStorage().beginTransaction();
ctx->temporary_directory_lock = std::move(lock);

View File

@ -2096,7 +2096,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()};
auto [dst_part, part_lock] = cloneAndLoadDataPart(
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(
src_part,
TMP_PREFIX,
dst_part_info,
@ -2207,7 +2207,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
.copy_instead_of_hardlink = getSettings()->always_use_copy_instead_of_hardlinks,
};
auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPart(
auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(
src_part,
TMP_PREFIX,
dst_part_info,

View File

@ -2788,7 +2788,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry)
auto obtain_part = [&] (PartDescriptionPtr & part_desc)
{
/// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPart(OnSameDisk) will do full copy.
/// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPartOnSameDisk will do full copy.
/// It's okay to check the setting for current table and disk for the source table, because src and dst part are on the same disk.
bool prefer_fetch_from_other_replica = !part_desc->replica.empty() && storage_settings_ptr->allow_remote_fs_zero_copy_replication
&& part_desc->src_table_part && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport();
@ -2807,7 +2807,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(LogEntry & entry)
.copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || ((our_zero_copy_enabled || source_zero_copy_enabled) && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport()),
.metadata_version_to_write = metadata_snapshot->getMetadataVersion()
};
auto [res_part, temporary_part_lock] = cloneAndLoadDataPart(
auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk(
part_desc->src_table_part,
TMP_PREFIX + "clone_",
part_desc->new_part_info,
@ -4888,7 +4888,7 @@ bool StorageReplicatedMergeTree::fetchPart(
.keep_metadata_version = true,
};
auto [cloned_part, lock] = cloneAndLoadDataPart(
auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(
part_to_clone,
"tmp_clone_",
part_info,
@ -8078,14 +8078,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication
|| dynamic_cast<const MergeTreeData *>(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication;
IDataPartStorage::ClonePartParams clone_params
{
.copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()),
.metadata_version_to_write = metadata_snapshot->getMetadataVersion()
};
auto [dst_part, part_lock] = cloneAndLoadDataPart(
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(
src_part,
TMP_PREFIX,
dst_part_info,
@ -8093,10 +8091,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
clone_params,
query_context->getReadSettings(),
query_context->getWriteSettings());
dst_parts.emplace_back(std::move(dst_part));
dst_parts_locks.emplace_back(std::move(part_lock));
src_parts.emplace_back(src_part);
dst_parts.emplace_back(dst_part);
dst_parts_locks.emplace_back(std::move(part_lock));
ephemeral_locks.emplace_back(std::move(*lock));
block_id_paths.emplace_back(block_id_path);
part_checksums.emplace_back(hash_hex);
@ -8349,7 +8346,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
.copy_instead_of_hardlink = storage_settings_ptr->always_use_copy_instead_of_hardlinks || (zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport()),
.metadata_version_to_write = dest_metadata_snapshot->getMetadataVersion()
};
auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPart(
auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(
src_part,
TMP_PREFIX,
dst_part_info,

View File

@ -2121,11 +2121,13 @@ def main() -> int:
pr_info,
dump_to_file=True,
)
update_mergeable_check(
commit,
pr_info,
job_report.check_name or _get_ext_check_name(args.job_name),
)
if not pr_info.is_merge_queue():
# in the merge queue mergeable status must be set only in FinishCheck (last job in wf)
update_mergeable_check(
commit,
pr_info,
job_report.check_name or _get_ext_check_name(args.job_name),
)
print(f"Job report url: [{check_url}]")
prepared_events = prepare_tests_results_for_clickhouse(

View File

@ -28,21 +28,22 @@ def main():
statuses = get_commit_filtered_statuses(commit)
trigger_mergeable_check(commit, statuses)
statuses = [s for s in statuses if s.context == CI_STATUS_NAME]
if not statuses:
return
# Take the latest status
status = statuses[-1]
if status.state == PENDING:
post_commit_status(
commit,
SUCCESS,
status.target_url,
"All checks finished",
CI_STATUS_NAME,
pr_info,
dump_to_file=True,
)
if not pr_info.is_merge_queue():
statuses = [s for s in statuses if s.context == CI_STATUS_NAME]
if not statuses:
return
# Take the latest status
status = statuses[-1]
if status.state == PENDING:
post_commit_status(
commit,
SUCCESS,
status.target_url,
"All checks finished",
CI_STATUS_NAME,
pr_info,
dump_to_file=True,
)
if __name__ == "__main__":

View File

@ -2,14 +2,70 @@
"""Script for automatic sync PRs handling in private repos"""
import argparse
import sys
from get_robot_token import get_best_robot_token
from pr_info import PRInfo
from github_helper import GitHub
from commit_status_helper import get_commit, post_commit_status
from report import FAILURE, SUCCESS
def parse_args() -> argparse.Namespace:
parser = argparse.ArgumentParser(
formatter_class=argparse.ArgumentDefaultsHelpFormatter,
description="Script for handling sync PRs",
)
parser.add_argument(
"--merge",
action="store_true",
help="merge sync pr",
)
parser.add_argument(
"--status",
action="store_true",
help="check and set sync pr status",
)
args = parser.parse_args()
return args
def merge_sync_pr(gh, sync_pr):
if not sync_pr:
print("Sync PR not found - exiting")
return
if sync_pr.state == "closed":
print(f"Sync PR [{sync_pr.number}] already closed - exiting")
sys.exit(0)
if sync_pr.state != "open":
print(
f"WARNING: Unknown Sync PR [{sync_pr.number}] state [{sync_pr.state}] - exiting"
)
sys.exit(0)
print(f"Trying to merge Sync PR [{sync_pr.number}]")
if sync_pr.draft:
gh.toggle_pr_draft(sync_pr)
sync_pr.merge()
def set_sync_status(gh, pr_info, sync_pr):
if not sync_pr or not sync_pr.mergeable:
post_commit_status(
get_commit(gh, pr_info.sha), FAILURE, "", "Sync PR failure", "A Sync"
)
else:
post_commit_status(get_commit(gh, pr_info.sha), SUCCESS, "", "", "A Sync")
def main():
args = parse_args()
assert args.merge ^ args.status
gh = GitHub(get_best_robot_token())
pr_info = PRInfo()
@ -19,27 +75,20 @@ def main():
query=f"head:sync-upstream/pr/{pr_info.merged_pr} org:ClickHouse type:pr",
repo="ClickHouse/clickhouse-private",
)
sync_pr = None
if len(prs) > 1:
print(f"WARNING: More than one PR found [{prs}] - exiting")
sys.exit(0)
if len(prs) == 0:
elif len(prs) == 0:
print("WARNING: No Sync PR found")
sys.exit(0)
else:
sync_pr = prs[0]
pr = prs[0]
if pr.state == "closed":
print(f"Sync PR [{pr.number}] already closed - exiting")
sys.exit(0)
if pr.state != "open":
print(f"WARNING: Unknown Sync PR [{pr.number}] state [{pr.state}] - exiting")
sys.exit(0)
print(f"Trying to merge Sync PR [{pr.number}]")
if pr.draft:
gh.toggle_pr_draft(pr)
pr.merge()
if args.merge:
merge_sync_pr(gh, sync_pr)
elif args.status:
set_sync_status(gh, pr_info, sync_pr)
if __name__ == "__main__":

View File

@ -1,17 +0,0 @@
<clickhouse>
<remote_servers>
<test_cluster>
<shard>
<internal_replication>true</internal_replication>
<replica>
<host>replica1</host>
<port>9000</port>
</replica>
<replica>
<host>replica2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster>
</remote_servers>
</clickhouse>

View File

@ -1,187 +0,0 @@
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
replica1 = cluster.add_instance(
"replica1", with_zookeeper=True, main_configs=["configs/remote_servers.xml"]
)
replica2 = cluster.add_instance(
"replica2", with_zookeeper=True, main_configs=["configs/remote_servers.xml"]
)
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
except Exception as ex:
print(ex)
finally:
cluster.shutdown()
def cleanup(nodes):
for node in nodes:
node.query("DROP TABLE IF EXISTS source SYNC")
node.query("DROP TABLE IF EXISTS destination SYNC")
def create_source_table(node, table_name, replicated):
replica = node.name
engine = (
f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')"
if replicated
else "MergeTree()"
)
node.query_with_retry(
"""
ATTACH TABLE {table_name} UUID 'cf712b4f-2ca8-435c-ac23-c4393efe52f7'
(
price UInt32,
date Date,
postcode1 LowCardinality(String),
postcode2 LowCardinality(String),
type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4),
is_new UInt8,
duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2),
addr1 String,
addr2 String,
street LowCardinality(String),
locality LowCardinality(String),
town LowCardinality(String),
district LowCardinality(String),
county LowCardinality(String)
)
ENGINE = {engine}
ORDER BY (postcode1, postcode2, addr1, addr2)
SETTINGS disk = disk(type = web, endpoint = 'https://raw.githubusercontent.com/ClickHouse/web-tables-demo/main/web/')
""".format(
table_name=table_name, engine=engine
)
)
def create_destination_table(node, table_name, replicated):
replica = node.name
engine = (
f"ReplicatedMergeTree('/clickhouse/tables/1/{table_name}', '{replica}')"
if replicated
else "MergeTree()"
)
node.query_with_retry(
"""
CREATE TABLE {table_name}
(
price UInt32,
date Date,
postcode1 LowCardinality(String),
postcode2 LowCardinality(String),
type Enum8('other' = 0, 'terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4),
is_new UInt8,
duration Enum8('unknown' = 0, 'freehold' = 1, 'leasehold' = 2),
addr1 String,
addr2 String,
street LowCardinality(String),
locality LowCardinality(String),
town LowCardinality(String),
district LowCardinality(String),
county LowCardinality(String)
)
ENGINE = {engine}
ORDER BY (postcode1, postcode2, addr1, addr2)
""".format(
table_name=table_name, engine=engine
)
)
def test_both_mergtree(start_cluster):
create_source_table(replica1, "source", False)
create_destination_table(replica1, "destination", False)
replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source")
assert_eq_with_retry(
replica1,
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC",
replica1.query(
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"
),
)
assert_eq_with_retry(
replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH"
)
cleanup([replica1])
def test_all_replicated(start_cluster):
create_source_table(replica1, "source", True)
create_destination_table(replica1, "destination", True)
create_destination_table(replica2, "destination", True)
replica1.query("SYSTEM SYNC REPLICA destination")
replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source")
assert_eq_with_retry(
replica1,
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC",
replica1.query(
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"
),
)
assert_eq_with_retry(
replica1,
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC",
replica2.query(
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC"
),
)
assert_eq_with_retry(
replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH"
)
assert_eq_with_retry(
replica2, f"SELECT town from destination LIMIT 1", "SCARBOROUGH"
)
cleanup([replica1, replica2])
def test_only_destination_replicated(start_cluster):
create_source_table(replica1, "source", False)
create_destination_table(replica1, "destination", True)
create_destination_table(replica2, "destination", True)
replica1.query("SYSTEM SYNC REPLICA destination")
replica1.query(f"ALTER TABLE destination ATTACH PARTITION tuple() FROM source")
assert_eq_with_retry(
replica1,
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC",
replica1.query(
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC"
),
)
assert_eq_with_retry(
replica1,
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM source GROUP BY year ORDER BY year ASC",
replica2.query(
f"SELECT toYear(date) AS year,round(avg(price)) AS price,bar(price, 0, 1000000, 80) FROM destination GROUP BY year ORDER BY year ASC"
),
)
assert_eq_with_retry(
replica1, f"SELECT town from destination LIMIT 1", "SCARBOROUGH"
)
assert_eq_with_retry(
replica2, f"SELECT town from destination LIMIT 1", "SCARBOROUGH"
)
cleanup([replica1, replica2])

View File

@ -31,7 +31,7 @@ def test_distibuted_settings(start_cluster):
DETACH TABLE dist_1;
"""
)
assert "flush_on_detach = 1" in node.query("SHOW CREATE dist_1")
assert "flush_on_detach = true" in node.query("SHOW CREATE dist_1")
# flush_on_detach=true, so data_1 should have 1 row
assert int(node.query("SELECT count() FROM data_1")) == 1

View File

@ -5,7 +5,6 @@ import string
import threading
import time
from multiprocessing.dummy import Pool
from helpers.test_tools import assert_eq_with_retry
import pytest
from helpers.client import QueryRuntimeException
@ -1746,9 +1745,9 @@ def test_move_while_merge(start_cluster):
node1.query(f"DROP TABLE IF EXISTS {name} SYNC")
def test_move_across_policies_work_for_attach_not_work_for_move(start_cluster):
def test_move_across_policies_does_not_work(start_cluster):
try:
name = "test_move_across_policies_work_for_attach_not_work_for_move"
name = "test_move_across_policies_does_not_work"
node1.query(
"""
@ -1784,18 +1783,25 @@ def test_move_across_policies_work_for_attach_not_work_for_move(start_cluster):
except QueryRuntimeException:
"""All parts of partition 'all' are already on disk 'jbod2'."""
node1.query(
"""ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format(
name=name
)
)
assert_eq_with_retry(
node1,
"""SELECT * FROM {name}2""".format(name=name),
with pytest.raises(
QueryRuntimeException,
match=".*because disk does not belong to storage policy.*",
):
node1.query(
"""SELECT * FROM {name}""".format(name=name),
),
)
"""ALTER TABLE {name}2 ATTACH PARTITION tuple() FROM {name}""".format(
name=name
)
)
with pytest.raises(
QueryRuntimeException,
match=".*because disk does not belong to storage policy.*",
):
node1.query(
"""ALTER TABLE {name}2 REPLACE PARTITION tuple() FROM {name}""".format(
name=name
)
)
with pytest.raises(
QueryRuntimeException,
@ -1807,6 +1813,10 @@ def test_move_across_policies_work_for_attach_not_work_for_move(start_cluster):
)
)
assert node1.query(
"""SELECT * FROM {name}""".format(name=name)
).splitlines() == ["1"]
finally:
node1.query(f"DROP TABLE IF EXISTS {name} SYNC")
node1.query(f"DROP TABLE IF EXISTS {name}2 SYNC")

View File

@ -0,0 +1,16 @@
<test>
<create_query>CREATE TABLE test_table_1(id UInt64, value String) ENGINE=MergeTree ORDER BY id</create_query>
<create_query>CREATE TABLE test_table_2(id UInt64, value String) ENGINE=MergeTree ORDER BY id</create_query>
<fill_query>INSERT INTO test_table_1 SELECT number, number FROM numbers(5000000)</fill_query>
<fill_query>INSERT INTO test_table_2 SELECT number, number FROM numbers(5000000)</fill_query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 AND rhs.id = 5 FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE lhs.id = 5 FORMAT Null</query>
<query>SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id WHERE rhs.id = 5 FORMAT Null</query>
<drop_query>DROP TABLE test_table_1</drop_query>
<drop_query>DROP TABLE test_table_2</drop_query>
</test>

View File

@ -180,12 +180,14 @@ Filter column: notEquals(__table1.number, 1_UInt8)
> one condition of filter is pushed down before INNER JOIN
Join
Join
Filter column: notEquals(number, 1)
Filter column: and(notEquals(number, 1), notEquals(number, 2))
Join
Filter column: and(notEquals(b, 2), notEquals(b, 1))
> (analyzer) one condition of filter is pushed down before INNER JOIN
Join
Join
Filter column: notEquals(__table1.number, 1_UInt8)
Filter column: and(notEquals(__table1.number, 1_UInt8), notEquals(__table1.number, 2_UInt8))
Filter column: and(notEquals(__table2.b, 2_UInt8), notEquals(__table2.b, 1_UInt8))
3 3
> filter is pushed down before UNION
Union

View File

@ -248,14 +248,14 @@ $CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -q "
select number as a, r.b from numbers(4) as l any inner join (
select number + 2 as b from numbers(3)
) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" |
grep -o "Join\|Filter column: notEquals(number, 1)"
grep -o "Join\|Filter column: and(notEquals(number, 1), notEquals(number, 2))\|Filter column: and(notEquals(b, 2), notEquals(b, 1))"
echo "> (analyzer) one condition of filter is pushed down before INNER JOIN"
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -q "
explain actions = 1
select number as a, r.b from numbers(4) as l any inner join (
select number + 2 as b from numbers(3)
) as r on a = r.b where a != 1 and b != 2 settings enable_optimize_predicate_expression = 0" |
grep -o "Join\|Filter column: notEquals(__table1.number, 1_UInt8)"
grep -o "Join\|Filter column: and(notEquals(__table1.number, 1_UInt8), notEquals(__table1.number, 2_UInt8))\|Filter column: and(notEquals(__table2.b, 2_UInt8), notEquals(__table2.b, 1_UInt8))"
$CLICKHOUSE_CLIENT -q "
select number as a, r.b from numbers(4) as l any inner join (
select number + 2 as b from numbers(3)

View File

@ -0,0 +1,147 @@
import pty
import os
import shlex
import time
import multiprocessing
COMPLETION_TIMEOUT_SECONDS = 30
DEBUG_LOG = os.path.join(
os.environ["CLICKHOUSE_TMP"],
os.path.basename(os.path.abspath(__file__)).strip(".python") + ".debuglog",
)
def run_with_timeout(func, args, timeout):
process = multiprocessing.Process(target=func, args=args)
process.start()
process.join(timeout)
if process.is_alive():
process.terminate()
print("Timeout")
def test_completion(program, argv, comp_word):
comp_begin = comp_word[:-3]
shell_pid, master = pty.fork()
if shell_pid == 0:
os.execv(program, argv)
else:
try:
debug_log_fd = open(DEBUG_LOG, "a")
output_b = os.read(master, 4096)
output = output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
while not ":)" in output:
output_b = os.read(master, 4096)
output += output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
os.write(master, b"SET " + bytes(comp_begin.encode()))
output_b = os.read(master, 4096)
output = output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
while not comp_begin in output:
output_b = os.read(master, 4096)
output += output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
time.sleep(0.01)
os.write(master, b"\t")
output_b = os.read(master, 4096)
output = output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
# fail fast if there is a bell character in the output,
# meaning no concise completion is found
if "\x07" in output:
print(f"{comp_word}: FAIL")
return
while not comp_word in output:
output_b = os.read(master, 4096)
output += output_b.decode()
debug_log_fd.write(repr(output_b) + "\n")
debug_log_fd.flush()
print(f"{comp_word}: OK")
finally:
os.close(master)
debug_log_fd.close()
client_compwords_positive = [
# system.functions
"concatAssumeInjective",
# system.table_engines
"ReplacingMergeTree",
# system.formats
"JSONEachRow",
# system.table_functions
"clusterAllReplicas",
# system.data_type_families
"SimpleAggregateFunction",
# system.settings
"max_concurrent_queries_for_all_users",
# system.clusters
"test_shard_localhost",
# system.macros
"default_path_test",
# system.storage_policies, egh not uniq
"default",
# system.aggregate_function_combinators
"uniqCombined64ForEach",
# FIXME: one may add separate case for suggestion_limit
# system.databases
"system",
# system.tables
"aggregate_function_combinators",
# system.columns
"primary_key_bytes_in_memory_allocated",
# system.dictionaries
# FIXME: none
]
local_compwords_positive = [
# system.functions
"concatAssumeInjective",
# system.table_engines
"ReplacingMergeTree",
# system.formats
"JSONEachRow",
# system.table_functions
"clusterAllReplicas",
# system.data_type_families
"SimpleAggregateFunction",
]
if __name__ == "__main__":
print("# clickhouse-client")
clickhouse_client = os.environ["CLICKHOUSE_CLIENT"]
args = shlex.split(clickhouse_client)
args.append("--wait_for_suggestions_to_load")
[
run_with_timeout(
test_completion, [args[0], args, comp_word], COMPLETION_TIMEOUT_SECONDS
)
for comp_word in client_compwords_positive
]
print("# clickhouse-local")
clickhouse_local = os.environ["CLICKHOUSE_LOCAL"]
args = shlex.split(clickhouse_local)
args.append("--wait_for_suggestions_to_load")
[
run_with_timeout(
test_completion, [args[0], args, comp_word], COMPLETION_TIMEOUT_SECONDS
)
for comp_word in local_compwords_positive
]

View File

@ -1,142 +1,8 @@
#!/usr/bin/env bash
# Tags: long, no-ubsan
# Tags: long
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
SCRIPT_PATH="$CURDIR/$CLICKHOUSE_TEST_UNIQUE_NAME.generated-expect"
# NOTE: database = $CLICKHOUSE_DATABASE is superfluous
function test_completion_word()
{
local w=$1 && shift
local w_len=${#w}
local compword_begin=${w:0:$((w_len-3))}
local compword_end=${w:$((w_len-3))}
# NOTE:
# - here and below you should escape variables of the expect.
# - you should not use "expect <<..." since in this case timeout/eof will
# not work (I guess due to attached stdin)
# TODO: get build sanitizer and debug/release info to dynamically change test
# like here timeout 120 seconds is too big for release build
# but ok for sanitizer builds
cat > "$SCRIPT_PATH" << EOF
# NOTE: log will be appended
exp_internal -f $CLICKHOUSE_TMP/$(basename "${BASH_SOURCE[0]}").debuglog 0
# NOTE: when expect have EOF on stdin it also closes stdout, so let's reopen it
# again for logging
set stdout_channel [open "/dev/stdout" w]
log_user 0
set timeout 120
match_max 100000
expect_after {
# Do not ignore eof from expect
-i \$any_spawn_id eof { exp_continue }
# A default timeout action is to do nothing, change it to fail
-i \$any_spawn_id timeout { exit 1 }
}
spawn bash -c "$* --highlight 0"
expect ":) "
# Make a query
send -- "SET $compword_begin"
expect "SET $compword_begin"
# Wait for suggestions to load, they are loaded in background
set is_done 0
set timeout 1
while {\$is_done == 0} {
send -- "\\t"
expect {
"$compword_begin$compword_end" {
puts \$stdout_channel "$compword_begin$compword_end: OK"
set is_done 1
}
default {
sleep 1
}
}
}
close \$stdout_channel
send -- "\\3\\4"
expect eof
EOF
# NOTE: run expect under timeout since there is while loop that is not
# limited with timeout.
#
# NOTE: cat is required to serialize stdout for expect (without this pipe
# it will reopen the file again, and the output will be mixed).
timeout 2m expect -f "$SCRIPT_PATH" | cat
}
# last 3 bytes will be completed,
# so take this in mind when you will update the list.
client_compwords_positive=(
# system.functions
concatAssumeInjective
# system.table_engines
ReplacingMergeTree
# system.formats
JSONEachRow
# system.table_functions
clusterAllReplicas
# system.data_type_families
SimpleAggregateFunction
# system.settings
max_concurrent_queries_for_all_users
# system.clusters
test_shard_localhost
# system.macros
default_path_test
# system.storage_policies, egh not uniq
default
# system.aggregate_function_combinators
uniqCombined64ForEach
# FIXME: one may add separate case for suggestion_limit
# system.databases
system
# system.tables
aggregate_function_combinators
# system.columns
primary_key_bytes_in_memory_allocated
# system.dictionaries
# FIXME: none
)
local_compwords_positive=(
# system.functions
concatAssumeInjective
# system.table_engines
ReplacingMergeTree
# system.formats
JSONEachRow
# system.table_functions
clusterAllReplicas
# system.data_type_families
SimpleAggregateFunction
)
echo "# clickhouse-client"
for w in "${client_compwords_positive[@]}"; do
test_completion_word "$w" "$CLICKHOUSE_CLIENT"
done
echo "# clickhouse-local"
for w in "${local_compwords_positive[@]}"; do
test_completion_word "$w" "$CLICKHOUSE_LOCAL"
done
rm -f "${SCRIPT_PATH:?}"
exit 0
python3 "$CURDIR"/01676_clickhouse_client_autocomplete.python

View File

@ -127,3 +127,14 @@ SELECT arrayMap(x -> splitByChar(toString(id), arrayMap(x -> toString(1), [NULL]
DROP TABLE test_table;
-- { echoOff }
SELECT
groupArray(number) AS counts,
arraySum(arrayMap(x -> (x + 1), counts)) as hello,
arrayMap(x -> (x / hello), counts) AS res
FROM numbers(1000000) FORMAT Null;
SELECT
arrayWithConstant(pow(10,6), 1) AS nums,
arrayMap(x -> x, nums) AS m,
arrayMap(x -> x + arraySum(m), m) AS res FORMAT Null;

View File

@ -1,2 +1,2 @@
A2193552DCF8A9F99AC35F86BC4D2FFD
SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS use_query_cache = 1
SELECT hex(encrypt(\'aes-128-ecb\', \'[HIDDEN]\')) SETTINGS use_query_cache = true

View File

@ -7,4 +7,4 @@
1
1 1
1 1
1 1
1 2

View File

@ -1,3 +1,5 @@
SET allow_experimental_analyzer = 1;
DROP TABLE IF EXISTS t1;
CREATE TABLE t1 (key UInt8) ENGINE = Memory;

View File

@ -5,33 +5,38 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# Copy backups/with_broken_part.zip into the disk named "backups".
SRC_BACKUP_DIR=$CURDIR/backups
SRC_BACKUP_FILENAME=with_broken_part.zip
# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk,
# returns the path to the backup relative to that disk.
function install_test_backup()
{
local test_backup_filename="$1"
local test_backup_path="$CURDIR/backups/${test_backup_filename}"
BACKUPS_DISK=backups
BACKUPS_DIR=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='$BACKUPS_DISK'")
local backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'")
if [ -z "$BACKUPS_DIR" ]; then
echo Disk \'$BACKUPS_DISK\' not found
exit 1
fi
if [ -z "${backups_disk_root}" ]; then
echo Disk \'${backups_disk_root}\' not found
exit 1
fi
BACKUP_FILENAME=$CLICKHOUSE_DATABASE/${SRC_BACKUP_FILENAME}
BACKUP_NAME="Disk('$BACKUPS_DISK', '$BACKUP_FILENAME')"
local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename}
mkdir -p "$(dirname "${install_path}")"
ln -s "${test_backup_path}" "${install_path}"
mkdir -p "$(dirname "$BACKUPS_DIR/$BACKUP_FILENAME")"
ln -s "$SRC_BACKUP_DIR/$SRC_BACKUP_FILENAME" "$BACKUPS_DIR/$BACKUP_FILENAME"
echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}"
}
backup_name="$(install_test_backup with_broken_part.zip)"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl"
# First try to restore with the setting `restore_broken_parts_as_detached` set to false.
$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME" 2>&1 | tr -d \\n | grep "data.bin doesn't exist" | grep "while restoring part all_2_2_0" > /dev/null && echo "OK" || echo "FAILED"
$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM Disk('backups', '${backup_name}')" 2>&1 | tr -d \\n | grep "data.bin doesn't exist" | grep "while restoring part all_2_2_0" > /dev/null && echo "OK" || echo "FAILED"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS tbl"
# Then try to restore with the setting `restore_broken_parts_as_detached` set to true.
$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM $BACKUP_NAME SETTINGS restore_broken_parts_as_detached = true" 2>/dev/null | awk -F '\t' '{print $2}'
$CLICKHOUSE_CLIENT --query "RESTORE TABLE default.tbl AS tbl FROM Disk('backups', '${backup_name}') SETTINGS restore_broken_parts_as_detached = true" 2>/dev/null | awk -F '\t' '{print $2}'
$CLICKHOUSE_CLIENT --multiquery <<EOF
SELECT * FROM tbl ORDER BY x;

View File

@ -8,6 +8,14 @@ SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND
2 2 2 2
3 3 3 33
\N \N \N \N
SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) ORDER BY t1.x;
2 2 2 2
3 3 3 33
\N \N \N \N
SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x;
2 2 2 2
3 3 3 33
\N \N \N \N
SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST;
2 2 2 2
\N \N \N \N

View File

@ -14,6 +14,9 @@ SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR (t1.x IS NULL AND t2.x IS NULL)) O
SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.x IS NULL AND t1.y <=> t2.y AND t2.x IS NULL) ORDER BY t1.x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) ORDER BY t1.x;
SELECT * FROM t1 JOIN t2 ON t1.x <=> t2.x AND (t1.x = t1.y OR t1.x IS NULL AND t1.y IS NULL) ORDER BY t1.x;
SELECT * FROM t1 JOIN t2 ON (t1.x = t2.x OR t1.x IS NULL AND t2.x IS NULL) AND t1.y <=> t2.y ORDER BY t1.x NULLS LAST;
SELECT * FROM t1 JOIN t2 ON (t1.x <=> t2.x OR t1.y <=> t2.y OR (t1.x IS NULL AND t1.y IS NULL AND t2.x IS NULL AND t2.y IS NULL)) ORDER BY t1.x NULLS LAST;

View File

@ -1,6 +1,5 @@
#!/usr/bin/env bash
# Tags: no-ordinary-database, no-replicated-database
# Tag no-ordinary-database: TO DO
# Tags: no-replicated-database
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh

View File

@ -0,0 +1,4 @@
RESTORED
2024-02-22 07:00:00 00
2024-02-22 07:00:01 11
2024-02-22 07:00:02 22

View File

@ -0,0 +1,40 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# Copies a test predefined backup from "/tests/queries/0_stateless/backups/" folder to the "backups" disk,
# returns the path to the backup relative to that disk.
function install_test_backup()
{
local test_backup_filename="$1"
local test_backup_path="$CURDIR/backups/${test_backup_filename}"
local backups_disk_root=$($CLICKHOUSE_CLIENT --query "SELECT path FROM system.disks WHERE name='backups'")
if [ -z "${backups_disk_root}" ]; then
echo Disk \'${backups_disk_root}\' not found
exit 1
fi
local install_path=${backups_disk_root}/${CLICKHOUSE_DATABASE}/${test_backup_filename}
mkdir -p "$(dirname "${install_path}")"
ln -s "${test_backup_path}" "${install_path}"
echo "${CLICKHOUSE_DATABASE}/${test_backup_filename}"
}
backup_name="$(install_test_backup old_backup_with_matview_inner_table_metadata.zip)"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mv"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS src"
db="$CLICKHOUSE_DATABASE"
${CLICKHOUSE_CLIENT} -q "RESTORE DATABASE mydb AS ${db} FROM Disk('backups', '${backup_name}') SETTINGS allow_different_database_def=true" | grep -o "RESTORED"
${CLICKHOUSE_CLIENT} -q "SELECT toDateTime(timestamp, 'UTC') AS ts, c12 FROM mv ORDER BY ts"
$CLICKHOUSE_CLIENT --query "DROP TABLE mv"
$CLICKHOUSE_CLIENT --query "DROP TABLE src"

View File

@ -0,0 +1,7 @@
# foo
foo
# !foo
# bar
bar
# defaults
bam

View File

@ -0,0 +1,20 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
tmp_file="$CUR_DIR/$CLICKHOUSE_DATABASE.txt"
echo '# foo'
$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -n -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select * from input('x String') format LineAsString" <<<foo
cat "$tmp_file"
echo '# !foo'
$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -n -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select * from input('x String') where x != 'foo' format LineAsString" <<<foo
cat "$tmp_file"
echo '# bar'
$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -n -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select y from input('x String, y String') format TSV" <<<$'foo\tbar'
cat "$tmp_file"
echo '# defaults'
$CLICKHOUSE_LOCAL --input_format_tsv_empty_as_default=1 --engine_file_truncate_on_insert=1 -n -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select y from input('x String, y String DEFAULT \\'bam\\'') format TSV" <<<$'foo\t'
cat "$tmp_file"
rm -f "${tmp_file:?}"

View File

@ -0,0 +1,35 @@
-- Custom settings from system.settings
custom_f1 Bool_0 Custom
custom_f2 Bool_0 Custom
custom_f3 Bool_0 Custom
custom_n0 UInt64_0 Custom
custom_n1 UInt64_1 Custom
custom_t1 Bool_1 Custom
custom_t2 Bool_1 Custom
custom_t3 Bool_1 Custom
-- Custom settings via getSetting()
custom_f1 false Bool
custom_f2 false Bool
custom_f3 false Bool
custom_n0 0 UInt8
custom_n1 1 UInt8
custom_t1 true Bool
custom_t2 true Bool
custom_t3 true Bool
-- Built-in settings
async_insert 0 Bool
async_insert false Bool
async_insert 0 Bool
async_insert false Bool
async_insert 0 Bool
async_insert false Bool
async_insert 0 Bool
async_insert false Bool
async_insert 1 Bool
async_insert true Bool
async_insert 1 Bool
async_insert true Bool
async_insert 1 Bool
async_insert true Bool
async_insert 1 Bool
async_insert true Bool

View File

@ -0,0 +1,65 @@
-- Custom settings must remember their types - whether it's a boolean or an integer.
-- Different ways to set a boolean.
SET custom_f1 = false;
SET custom_f2 = False;
SET custom_f3 = FALSE;
SET custom_n0 = 0;
SET custom_n1 = 1;
SET custom_t1 = true;
SET custom_t2 = True;
SET custom_t3 = TRUE;
SELECT '-- Custom settings from system.settings';
SELECT name, value, type FROM system.settings WHERE startsWith(name, 'custom_') ORDER BY name;
SELECT '-- Custom settings via getSetting()';
SELECT 'custom_f1' AS name, getSetting(name) AS value, toTypeName(value);
SELECT 'custom_f2' AS name, getSetting(name) AS value, toTypeName(value);
SELECT 'custom_f3' AS name, getSetting(name) AS value, toTypeName(value);
SELECT 'custom_n0' AS name, getSetting(name) AS value, toTypeName(value);
SELECT 'custom_n1' AS name, getSetting(name) AS value, toTypeName(value);
SELECT 'custom_t1' AS name, getSetting(name) AS value, toTypeName(value);
SELECT 'custom_t2' AS name, getSetting(name) AS value, toTypeName(value);
SELECT 'custom_t3' AS name, getSetting(name) AS value, toTypeName(value);
-- Built-in settings have hardcoded types.
SELECT '-- Built-in settings';
SET async_insert = false;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);
SET async_insert = False;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);
SET async_insert = FALSE;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);
SET async_insert = 0;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);
SET async_insert = 1;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);
SET async_insert = true;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);
SET async_insert = True;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);
SET async_insert = TRUE;
SELECT name, value, type FROM system.settings WHERE name = 'async_insert';
SELECT 'async_insert' AS name, getSetting(name) AS value, toTypeName(value);

View File

@ -0,0 +1,5 @@
BACKUP_CREATED
RESTORED
2
80
-12345

View File

@ -0,0 +1,56 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} -nm --query "
DROP TABLE IF EXISTS tbl;
DROP TABLE IF EXISTS tbl2;
CREATE TABLE tbl (a Int32) ENGINE = MergeTree() ORDER BY tuple();
INSERT INTO tbl VALUES (2), (80), (-12345);
"
function start_async()
{
local command="$1"
local first_column="s/^\([^\t]*\)\t.*/\1/"
echo $(${CLICKHOUSE_CLIENT} --query "$command" | sed "${first_column}")
}
function wait_status()
{
local operation_id="$1"
local expected_status="$2"
local timeout=60
local start=$EPOCHSECONDS
while true; do
local current_status=$(${CLICKHOUSE_CLIENT} --query "SELECT status FROM system.backups WHERE id='${operation_id}'")
if [ "${current_status}" == "${expected_status}" ]; then
echo "${current_status}"
break
fi
if ((EPOCHSECONDS-start > timeout )); then
echo "Timeout while waiting for operation ${operation_id} to come to status ${expected_status}. The current status is ${current_status}."
exit 1
fi
sleep 0.1
done
}
# Making a backup.
backup_name="Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}')"
backup_operation_id=$(start_async "BACKUP TABLE tbl TO ${backup_name} ASYNC")
wait_status ${backup_operation_id} "BACKUP_CREATED"
# Restoring from that backup.
restore_operation_id=$(start_async "RESTORE TABLE tbl AS tbl2 FROM ${backup_name} ASYNC")
wait_status ${restore_operation_id} "RESTORED"
# Check the result of that restoration.
${CLICKHOUSE_CLIENT} --query "SELECT * FROM tbl2"
${CLICKHOUSE_CLIENT} -nm --query "
DROP TABLE tbl;
DROP TABLE tbl2;
"

View File

@ -0,0 +1,710 @@
-- { echoOn }
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
Expression ((Project names + (Projection + )))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table1.id UInt64
__table1.value String
Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table1.id UInt64 : 3
ALIAS value :: 1 -> __table1.value String : 0
FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table2.id UInt64
__table2.value String
Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table2.id UInt64 : 3
ALIAS value :: 1 -> __table2.value String : 0
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
Expression ((Project names + (Projection + )))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table1.id UInt64
__table1.value String
Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table1.id UInt64 : 3
ALIAS value :: 1 -> __table1.value String : 0
FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table2.id UInt64
__table2.value String
Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table2.id UInt64 : 3
ALIAS value :: 1 -> __table2.value String : 0
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;
Expression ((Project names + (Projection + )))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: INNER
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table1.id UInt64
__table1.value String
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 3
ALIAS id :: 0 -> __table1.id UInt64 : 4
ALIAS value :: 1 -> __table1.value String : 0
FUNCTION equals(__table1.id : 4, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1
FUNCTION equals(__table1.id : 4, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2
FUNCTION and(equals(__table1.id, 5_UInt8) :: 2, equals(__table1.id, 6_UInt8) :: 1) -> and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) UInt8 : 3
Positions: 3 4 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table2.id UInt64
__table2.value String
Filter column: and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 3
ALIAS id :: 0 -> __table2.id UInt64 : 4
ALIAS value :: 1 -> __table2.value String : 0
FUNCTION equals(__table2.id : 4, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1
FUNCTION equals(__table2.id : 4, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2
FUNCTION and(equals(__table2.id, 6_UInt8) :: 2, equals(__table2.id, 5_UInt8) :: 1) -> and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) UInt8 : 3
Positions: 3 4 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
Expression ((Project names + (Projection + )))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: LEFT
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table1.id UInt64
__table1.value String
Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table1.id UInt64 : 3
ALIAS value :: 1 -> __table1.value String : 0
FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table2.id UInt64
__table2.value String
Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table2.id UInt64 : 3
ALIAS value :: 1 -> __table2.value String : 0
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
Expression ((Project names + Projection))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Filter ((WHERE + DROP unused columns after JOIN))
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT :: 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: LEFT
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table1.id UInt64 : 2
ALIAS value :: 1 -> __table1.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table2.id UInt64 : 2
ALIAS value :: 1 -> __table2.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
Expression ((Project names + Projection))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Filter ((WHERE + DROP unused columns after JOIN))
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2
INPUT :: 3 -> __table2.id UInt64 : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: RIGHT
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table1.id UInt64 : 2
ALIAS value :: 1 -> __table1.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table2.id UInt64 : 2
ALIAS value :: 1 -> __table2.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
Expression ((Project names + (Projection + )))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: RIGHT
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table1.id UInt64
__table1.value String
Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table1.id UInt64 : 3
ALIAS value :: 1 -> __table1.value String : 0
FUNCTION equals(__table1.id : 3, 5_UInt8 :: 2) -> equals(__table1.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table2.id UInt64
__table2.value String
Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
ALIAS id :: 0 -> __table2.id UInt64 : 3
ALIAS value :: 1 -> __table2.value String : 0
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1
Positions: 1 3 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
Expression ((Project names + Projection))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Filter ((WHERE + DROP unused columns after JOIN))
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Filter column: equals(__table1.id, 5_UInt8) (removed)
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2
INPUT :: 3 -> __table2.id UInt64 : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: FULL
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table1.id UInt64 : 2
ALIAS value :: 1 -> __table1.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table2.id UInt64 : 2
ALIAS value :: 1 -> __table2.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
Expression ((Project names + Projection))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Filter ((WHERE + DROP unused columns after JOIN))
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Filter column: equals(__table2.id, 5_UInt8) (removed)
Actions: INPUT :: 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
FUNCTION equals(__table2.id : 3, 5_UInt8 :: 4) -> equals(__table2.id, 5_UInt8) UInt8 : 5
Positions: 5 0 1 2 3
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: FULL
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table1.id UInt64 : 2
ALIAS value :: 1 -> __table1.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table2.id UInt64 : 2
ALIAS value :: 1 -> __table2.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
5 5 5 5
SELECT '--';
--
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;
Expression ((Project names + Projection))
Header: id UInt64
rhs.id UInt64
value String
rhs.value String
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT : 1 -> __table1.value String : 1
INPUT : 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
ALIAS __table1.id :: 0 -> id UInt64 : 4
ALIAS __table1.value :: 1 -> value String : 0
ALIAS __table2.value :: 2 -> rhs.value String : 1
ALIAS __table2.id :: 3 -> rhs.id UInt64 : 2
Positions: 4 2 0 1
Filter ((WHERE + DROP unused columns after JOIN))
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed)
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6
FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4
FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5
Positions: 5 0 1 2 3
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String
__table2.value String
__table2.id UInt64
Type: FULL
Strictness: ALL
Algorithm: HashJoin
Clauses: [(__table1.id) = (__table2.id)]
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table1.id UInt64
__table1.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table1.id UInt64 : 2
ALIAS value :: 1 -> __table1.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
Expression ((JOIN actions + Change column names to column identifiers))
Header: __table2.id UInt64
__table2.value String
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
ALIAS id :: 0 -> __table2.id UInt64 : 2
ALIAS value :: 1 -> __table2.value String : 0
Positions: 2 0
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
ReadType: Default
Parts: 1
Granules: 1
SELECT '--';
--
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;

View File

@ -0,0 +1,131 @@
SET allow_experimental_analyzer = 1;
SET optimize_move_to_prewhere = 0;
DROP TABLE IF EXISTS test_table_1;
CREATE TABLE test_table_1
(
id UInt64,
value String
) ENGINE=MergeTree ORDER BY id;
CREATE TABLE test_table_2
(
id UInt64,
value String
) ENGINE=MergeTree ORDER BY id;
INSERT INTO test_table_1 SELECT number, number FROM numbers(10);
INSERT INTO test_table_2 SELECT number, number FROM numbers(10);
-- { echoOn }
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs LEFT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs RIGHT JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE rhs.id = 5;
SELECT '--';
EXPLAIN header = 1, actions = 1
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;
SELECT '--';
SELECT lhs.id, rhs.id, lhs.value, rhs.value FROM test_table_1 AS lhs FULL JOIN test_table_2 AS rhs ON lhs.id = rhs.id
WHERE lhs.id = 5 AND rhs.id = 6;
-- { echoOff }
DROP TABLE test_table_1;
DROP TABLE test_table_2;

View File

@ -1,5 +1,6 @@
-- https://github.com/ClickHouse/ClickHouse/issues/29838
SET allow_experimental_analyzer=1;
SET distributed_foreground_insert=1;
CREATE TABLE first_table_lr
(

View File

@ -0,0 +1,78 @@
-- Bug 37909
SELECT
v_date AS vDate,
round(sum(v_share)) AS v_sum
FROM
(
WITH
(
SELECT rand() % 10000
) AS dummy_1,
(
SELECT rand() % 10000
) AS dummy_2,
(
SELECT rand() % 10000
) AS dummy_3,
_v AS
(
SELECT
xxHash64(rand()) % 100000 AS d_id,
toDate(parseDateTimeBestEffort('2022-01-01') + (rand() % 2600000)) AS v_date
FROM numbers(1000000)
ORDER BY d_id ASC
),
_i AS
(
SELECT xxHash64(rand()) % 40000 AS d_id
FROM numbers(1000000)
),
not_i AS
(
SELECT
NULL AS v_date,
d_id,
0 AS v_share
FROM _i
LIMIT 100
)
SELECT *
FROM
(
SELECT
d_id,
v_date,
v_share
FROM not_i
UNION ALL
SELECT
d_id,
v_date,
1 AS v_share
FROM
(
SELECT
d_id,
arrayJoin(groupArray(v_date)) AS v_date
FROM
(
SELECT
v_date,
d_id
FROM _v
UNION ALL
SELECT
NULL AS v_date,
d_id
FROM _i
)
GROUP BY d_id
)
)
WHERE (v_date >= '2022-05-08') AND (v_date <= '2022-06-07')
)
/* WHERE (v_date >= '2022-05-08') AND (v_date <= '2022-06-07') placing condition has same effect */
GROUP BY vDate
ORDER BY vDate ASC
SETTINGS allow_experimental_analyzer = 1; -- the query times out if allow_experimental_analyzer = 0

View File

@ -0,0 +1,5 @@
1 \N 1
1 \N 1
1 \N 1
1 \N 1
1 \N 1

View File

@ -0,0 +1,11 @@
{% for join_algorithm in ['default', 'full_sorting_merge', 'hash', 'partial_merge', 'grace_hash'] -%}
SET join_algorithm = '{{ join_algorithm }}';
SELECT *
FROM (SELECT 1 AS key) AS t1
JOIN (SELECT NULL, 1 AS key) AS t2
ON t1.key = t2.key
WHERE t1.key ORDER BY key;
{% endfor -%}

View File

@ -0,0 +1 @@
(1,'test') 1

View File

@ -0,0 +1,4 @@
SELECT
JSONExtract('{"a":1, "b":"test"}', 'Tuple(a UInt8, b String)') AS x,
x.a
SETTINGS allow_experimental_analyzer = 1;

View File

@ -183,6 +183,8 @@ CompiledExpressionCacheCount
ComplexKeyCache
ComplexKeyDirect
ComplexKeyHashed
Composable
composable
Config
ConnectionDetails
Const
@ -697,6 +699,7 @@ PCRE
PRCP
PREWHERE
PROCESSLIST
PROXYv
PSUN
PagerDuty
ParallelFormattingOutputFormatThreads