2018-10-16 19:00:05 +00:00
|
|
|
#pragma once
|
|
|
|
|
2018-11-10 20:09:07 +00:00
|
|
|
#include <Core/Block.h>
|
|
|
|
#include <Core/NamesAndTypes.h>
|
|
|
|
#include <Databases/IDatabase.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Interpreters/IdentifierSemantic.h>
|
|
|
|
#include <Interpreters/InDepthNodeVisitor.h>
|
|
|
|
#include <Interpreters/interpretSubquery.h>
|
2022-07-18 15:53:30 +00:00
|
|
|
#include <Interpreters/PreparedSets.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Parsers/ASTFunction.h>
|
2022-03-18 07:32:42 +00:00
|
|
|
#include <Parsers/ASTIdentifier.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Parsers/ASTSubquery.h>
|
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
|
|
|
#include <Parsers/IAST.h>
|
2021-09-16 17:40:42 +00:00
|
|
|
#include <Processors/Executors/CompletedPipelineExecutor.h>
|
2021-07-23 19:33:59 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2022-03-18 07:32:42 +00:00
|
|
|
#include <Processors/QueryPlan/QueryPlan.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2022-03-18 07:32:42 +00:00
|
|
|
#include <Storages/ColumnsDescription.h>
|
|
|
|
#include <Storages/ConstraintsDescription.h>
|
|
|
|
#include <Storages/IStorage.h>
|
2018-11-10 20:09:07 +00:00
|
|
|
|
2018-10-16 19:00:05 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
|
2020-02-25 18:10:48 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-09-09 10:03:13 +00:00
|
|
|
extern const int WRONG_GLOBAL_SUBQUERY;
|
2020-02-25 18:10:48 +00:00
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
class GlobalSubqueriesMatcher
|
2018-10-16 19:00:05 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-04-10 23:33:54 +00:00
|
|
|
struct Data : WithContext
|
2018-10-16 19:00:05 +00:00
|
|
|
{
|
2018-12-07 15:36:54 +00:00
|
|
|
size_t subquery_depth;
|
|
|
|
bool is_remote;
|
2022-04-07 11:43:49 +00:00
|
|
|
bool is_explain;
|
2020-03-10 19:36:17 +00:00
|
|
|
TemporaryTablesMapping & external_tables;
|
2022-07-18 15:53:30 +00:00
|
|
|
PreparedSetsPtr prepared_sets;
|
2018-12-07 15:36:54 +00:00
|
|
|
bool & has_global_subqueries;
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
Data(
|
|
|
|
ContextPtr context_,
|
|
|
|
size_t subquery_depth_,
|
|
|
|
bool is_remote_,
|
2022-04-07 11:43:49 +00:00
|
|
|
bool is_explain_,
|
2021-04-10 23:33:54 +00:00
|
|
|
TemporaryTablesMapping & tables,
|
2022-07-18 15:53:30 +00:00
|
|
|
PreparedSetsPtr prepared_sets_,
|
2021-04-10 23:33:54 +00:00
|
|
|
bool & has_global_subqueries_)
|
|
|
|
: WithContext(context_)
|
|
|
|
, subquery_depth(subquery_depth_)
|
|
|
|
, is_remote(is_remote_)
|
2022-04-07 11:43:49 +00:00
|
|
|
, is_explain(is_explain_)
|
2021-04-10 23:33:54 +00:00
|
|
|
, external_tables(tables)
|
2022-07-18 15:53:30 +00:00
|
|
|
, prepared_sets(prepared_sets_)
|
2021-04-10 23:33:54 +00:00
|
|
|
, has_global_subqueries(has_global_subqueries_)
|
|
|
|
{
|
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2019-04-10 17:41:19 +00:00
|
|
|
void addExternalStorage(ASTPtr & ast, bool set_alias = false)
|
2018-10-16 19:00:05 +00:00
|
|
|
{
|
2018-12-07 15:36:54 +00:00
|
|
|
/// With nondistributed queries, creating temporary tables does not make sense.
|
|
|
|
if (!is_remote)
|
|
|
|
return;
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2019-04-10 17:41:19 +00:00
|
|
|
bool is_table = false;
|
2021-08-03 11:18:47 +00:00
|
|
|
ASTPtr subquery_or_table_name; /// ASTTableIdentifier | ASTSubquery | ASTTableExpression
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2019-04-10 17:41:19 +00:00
|
|
|
if (const auto * ast_table_expr = ast->as<ASTTableExpression>())
|
2018-12-07 15:36:54 +00:00
|
|
|
{
|
2019-04-10 17:41:19 +00:00
|
|
|
subquery_or_table_name = ast_table_expr->subquery;
|
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
if (ast_table_expr->database_and_table_name)
|
|
|
|
{
|
2019-04-10 17:41:19 +00:00
|
|
|
subquery_or_table_name = ast_table_expr->database_and_table_name;
|
|
|
|
is_table = true;
|
2018-12-07 15:36:54 +00:00
|
|
|
}
|
|
|
|
}
|
2020-11-13 14:34:47 +00:00
|
|
|
else if (ast->as<ASTTableIdentifier>())
|
2021-08-03 11:18:47 +00:00
|
|
|
{
|
|
|
|
subquery_or_table_name = ast;
|
2019-04-10 17:41:19 +00:00
|
|
|
is_table = true;
|
2021-08-03 11:18:47 +00:00
|
|
|
}
|
2021-08-03 12:16:55 +00:00
|
|
|
else if (ast->as<ASTSubquery>())
|
|
|
|
{
|
|
|
|
subquery_or_table_name = ast;
|
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
if (!subquery_or_table_name)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::WRONG_GLOBAL_SUBQUERY, "Global subquery requires subquery or table name");
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2019-04-10 17:41:19 +00:00
|
|
|
if (is_table)
|
2018-12-07 15:36:54 +00:00
|
|
|
{
|
|
|
|
/// If this is already an external table, you do not need to add anything. Just remember its presence.
|
2020-03-11 19:10:55 +00:00
|
|
|
auto temporary_table_name = getIdentifierName(subquery_or_table_name);
|
2022-03-18 07:32:42 +00:00
|
|
|
bool exists_in_local_map = external_tables.contains(temporary_table_name);
|
2021-06-06 22:52:36 +00:00
|
|
|
bool exists_in_context = static_cast<bool>(getContext()->tryResolveStorageID(
|
|
|
|
StorageID("", temporary_table_name), Context::ResolveExternal));
|
2020-03-11 19:10:55 +00:00
|
|
|
if (exists_in_local_map || exists_in_context)
|
2018-12-07 15:36:54 +00:00
|
|
|
return;
|
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2022-03-18 07:32:42 +00:00
|
|
|
String alias = subquery_or_table_name->tryGetAlias();
|
|
|
|
String external_table_name;
|
|
|
|
if (alias.empty())
|
2018-12-07 15:36:54 +00:00
|
|
|
{
|
2022-03-18 07:32:42 +00:00
|
|
|
auto hash = subquery_or_table_name->getTreeHash();
|
|
|
|
external_table_name = fmt::format("_data_{}_{}", hash.first, hash.second);
|
2018-12-07 15:36:54 +00:00
|
|
|
}
|
2022-03-18 07:32:42 +00:00
|
|
|
else
|
|
|
|
external_table_name = alias;
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
/** We replace the subquery with the name of the temporary table.
|
|
|
|
* It is in this form, the request will go to the remote server.
|
|
|
|
* This temporary table will go to the remote server, and on its side,
|
|
|
|
* instead of doing a subquery, you just need to read it.
|
2022-03-18 07:32:42 +00:00
|
|
|
* TODO We can do better than using alias to name external tables
|
2018-12-07 15:36:54 +00:00
|
|
|
*/
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2020-10-26 15:49:00 +00:00
|
|
|
auto database_and_table_name = std::make_shared<ASTTableIdentifier>(external_table_name);
|
2019-04-10 17:41:19 +00:00
|
|
|
if (set_alias)
|
|
|
|
{
|
2020-10-26 15:49:00 +00:00
|
|
|
if (auto * table_name = subquery_or_table_name->as<ASTTableIdentifier>())
|
2019-04-10 17:41:19 +00:00
|
|
|
if (alias.empty())
|
|
|
|
alias = table_name->shortName();
|
|
|
|
database_and_table_name->setAlias(alias);
|
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2019-04-10 17:41:19 +00:00
|
|
|
if (auto * ast_table_expr = ast->as<ASTTableExpression>())
|
2018-12-07 15:36:54 +00:00
|
|
|
{
|
|
|
|
ast_table_expr->subquery.reset();
|
|
|
|
ast_table_expr->database_and_table_name = database_and_table_name;
|
|
|
|
|
|
|
|
ast_table_expr->children.clear();
|
|
|
|
ast_table_expr->children.emplace_back(database_and_table_name);
|
|
|
|
}
|
|
|
|
else
|
2019-04-10 17:41:19 +00:00
|
|
|
ast = database_and_table_name;
|
2018-12-07 15:36:54 +00:00
|
|
|
|
2022-03-18 07:32:42 +00:00
|
|
|
if (external_tables.contains(external_table_name))
|
|
|
|
return;
|
|
|
|
|
|
|
|
auto interpreter = interpretSubquery(subquery_or_table_name, getContext(), subquery_depth, {});
|
|
|
|
|
|
|
|
Block sample = interpreter->getSampleBlock();
|
|
|
|
NamesAndTypesList columns = sample.getNamesAndTypesList();
|
|
|
|
|
|
|
|
auto external_storage_holder = std::make_shared<TemporaryTableHolder>(
|
|
|
|
getContext(),
|
|
|
|
ColumnsDescription{columns},
|
|
|
|
ConstraintsDescription{},
|
|
|
|
nullptr,
|
|
|
|
/*create_for_global_subquery*/ true);
|
|
|
|
StoragePtr external_storage = external_storage_holder->getTable();
|
|
|
|
|
|
|
|
external_tables.emplace(external_table_name, external_storage_holder);
|
2020-11-21 08:25:45 +00:00
|
|
|
|
2022-03-18 07:32:42 +00:00
|
|
|
/// We need to materialize external tables immediately because reading from distributed
|
|
|
|
/// tables might generate local plans which can refer to external tables during index
|
|
|
|
/// analysis. It's too late to populate the external table via CreatingSetsTransform.
|
2022-04-07 11:43:49 +00:00
|
|
|
if (is_explain)
|
|
|
|
{
|
|
|
|
/// Do not materialize external tables if it's explain statement.
|
|
|
|
}
|
2023-04-25 18:14:08 +00:00
|
|
|
// else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries)
|
|
|
|
// {
|
|
|
|
// auto external_table = external_storage_holder->getTable();
|
|
|
|
// auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext());
|
|
|
|
// auto io = interpreter->execute();
|
|
|
|
// io.pipeline.complete(std::move(table_out));
|
|
|
|
// CompletedPipelineExecutor executor(io.pipeline);
|
|
|
|
// executor.execute();
|
|
|
|
// }
|
2020-11-21 08:25:45 +00:00
|
|
|
else
|
|
|
|
{
|
2023-04-25 18:14:08 +00:00
|
|
|
// auto & subquery_for_set = prepared_sets->getSubquery(external_table_name);
|
|
|
|
// subquery_for_set.createSource(*interpreter, external_storage);
|
|
|
|
auto key = subquery_or_table_name->getColumnName();
|
2023-05-24 17:53:37 +00:00
|
|
|
auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash());
|
2023-05-04 17:54:08 +00:00
|
|
|
|
|
|
|
if (!prepared_sets->getFuture(set_key))
|
|
|
|
{
|
|
|
|
SubqueryForSet subquery_for_set;
|
|
|
|
subquery_for_set.key = std::move(key);
|
|
|
|
subquery_for_set.table = std::move(external_storage);
|
|
|
|
subquery_for_set.createSource(*interpreter);
|
|
|
|
|
2023-05-24 17:53:37 +00:00
|
|
|
//std::cerr << reinterpret_cast<const void *>(prepared_sets.get()) << std::endl;
|
2023-05-04 17:54:08 +00:00
|
|
|
prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
prepared_sets->addStorageToSubquery(key, std::move(external_storage));
|
2020-11-21 08:25:45 +00:00
|
|
|
}
|
2018-12-07 15:36:54 +00:00
|
|
|
|
|
|
|
/** NOTE If it was written IN tmp_table - the existing temporary (but not external) table,
|
|
|
|
* then a new temporary table will be created (for example, _data1),
|
|
|
|
* and the data will then be copied to it.
|
|
|
|
* Maybe this can be avoided.
|
2018-10-16 19:00:05 +00:00
|
|
|
*/
|
2018-12-07 15:36:54 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2019-02-22 13:33:56 +00:00
|
|
|
static void visit(ASTPtr & ast, Data & data)
|
2018-12-07 15:36:54 +00:00
|
|
|
{
|
2019-03-11 13:22:51 +00:00
|
|
|
if (auto * t = ast->as<ASTFunction>())
|
2018-12-07 15:36:54 +00:00
|
|
|
visit(*t, ast, data);
|
2019-03-11 13:22:51 +00:00
|
|
|
if (auto * t = ast->as<ASTTablesInSelectQueryElement>())
|
2018-12-07 15:36:54 +00:00
|
|
|
visit(*t, ast, data);
|
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
static bool needChildVisit(ASTPtr &, const ASTPtr & child)
|
|
|
|
{
|
|
|
|
/// We do not go into subqueries.
|
2020-11-13 14:13:27 +00:00
|
|
|
return !child->as<ASTSelectQuery>();
|
2018-12-07 15:36:54 +00:00
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
private:
|
|
|
|
/// GLOBAL IN
|
|
|
|
static void visit(ASTFunction & func, ASTPtr &, Data & data)
|
|
|
|
{
|
2021-04-21 13:14:40 +00:00
|
|
|
if ((data.getContext()->getSettingsRef().prefer_global_in_and_join
|
|
|
|
&& (func.name == "in" || func.name == "notIn" || func.name == "nullIn" || func.name == "notNullIn"))
|
|
|
|
|| func.name == "globalIn" || func.name == "globalNotIn" || func.name == "globalNullIn" || func.name == "globalNotNullIn")
|
2018-10-16 19:00:05 +00:00
|
|
|
{
|
2020-04-11 21:28:04 +00:00
|
|
|
ASTPtr & ast = func.arguments->children[1];
|
|
|
|
|
2021-07-11 06:09:52 +00:00
|
|
|
/// Literal or function can use regular IN.
|
|
|
|
/// NOTE: We don't support passing table functions to IN.
|
|
|
|
if (ast->as<ASTLiteral>() || ast->as<ASTFunction>())
|
2020-04-11 21:28:04 +00:00
|
|
|
{
|
|
|
|
if (func.name == "globalIn")
|
|
|
|
func.name = "in";
|
2021-04-21 13:14:40 +00:00
|
|
|
else if (func.name == "globalNotIn")
|
2020-04-11 21:28:04 +00:00
|
|
|
func.name = "notIn";
|
2021-04-21 13:14:40 +00:00
|
|
|
else if (func.name == "globalNullIn")
|
|
|
|
func.name = "nullIn";
|
|
|
|
else if (func.name == "globalNotNullIn")
|
|
|
|
func.name = "notNullIn";
|
2020-04-11 21:28:04 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
data.addExternalStorage(ast);
|
2018-12-07 15:36:54 +00:00
|
|
|
data.has_global_subqueries = true;
|
|
|
|
}
|
|
|
|
}
|
2018-10-16 19:00:05 +00:00
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
/// GLOBAL JOIN
|
|
|
|
static void visit(ASTTablesInSelectQueryElement & table_elem, ASTPtr &, Data & data)
|
|
|
|
{
|
2021-04-21 13:14:40 +00:00
|
|
|
if (table_elem.table_join
|
2022-07-29 16:30:50 +00:00
|
|
|
&& (table_elem.table_join->as<ASTTableJoin &>().locality == JoinLocality::Global
|
2021-04-21 13:14:40 +00:00
|
|
|
|| data.getContext()->getSettingsRef().prefer_global_in_and_join))
|
2018-12-07 15:36:54 +00:00
|
|
|
{
|
2019-04-10 17:41:19 +00:00
|
|
|
data.addExternalStorage(table_elem.table_expression, true);
|
2018-12-07 15:36:54 +00:00
|
|
|
data.has_global_subqueries = true;
|
2018-10-16 19:00:05 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2018-12-07 15:36:54 +00:00
|
|
|
/// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr.
|
|
|
|
using GlobalSubqueriesVisitor = InDepthNodeVisitor<GlobalSubqueriesMatcher, false>;
|
|
|
|
|
2018-10-16 19:00:05 +00:00
|
|
|
}
|