Merge branch 'prepared_statement_cnchdev' into 'cnch-dev'

feat(optimizer@m-3000647987): Prepared Statement

See merge request dp/ClickHouse!19511
This commit is contained in:
Dao 2024-03-07 14:29:20 +08:00
parent d0a11969e9
commit a08037f299
111 changed files with 3442 additions and 316 deletions

View File

@ -0,0 +1,13 @@
#pragma once
#include <variant>
// helper type for the visitor #4
template <class... Ts>
struct overloaded : Ts...
{
using Ts::operator()...;
};
// explicit deduction guide (not needed as of C++20)
template <class... Ts>
overloaded(Ts...) -> overloaded<Ts...>;

View File

@ -56,6 +56,7 @@
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/RuntimeFilter/RuntimeFilterService.h>
#include <Interpreters/SQLBinding/SQLBindingCache.h>
@ -1833,6 +1834,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
Statistics::CacheManager::initialize(global_context);
BindingCacheManager::initializeGlobalBinding(global_context);
PlanCacheManager::initialize(global_context);
PreparedStatementManager::initialize(global_context);
Statistics::AutoStats::AutoStatisticsManager::initialize(global_context, global_context->getConfigRef());
}

View File

@ -80,6 +80,7 @@ public:
ColumnWithTypeAndName visitASTOrderByElement(ASTPtr & node, const Void &) override;
ColumnWithTypeAndName visitASTQuantifiedComparison(ASTPtr & node, const Void &) override;
ColumnWithTypeAndName visitASTTableColumnReference(ASTPtr & node, const Void &) override;
ColumnWithTypeAndName visitASTPreparedParameter(ASTPtr & node, const Void &) override;
ExprAnalyzerVisitor(ContextPtr context_, Analysis & analysis_, ScopePtr scope_, ExprAnalyzerOptions options_)
: context(std::move(context_))
@ -306,6 +307,12 @@ ColumnWithTypeAndName ExprAnalyzerVisitor::visitASTSubquery(ASTPtr & node, const
return {nullptr, type, node->getColumnName()};
}
ColumnWithTypeAndName ExprAnalyzerVisitor::visitASTPreparedParameter(ASTPtr & node, const Void &)
{
const auto & prepared_param = node->as<ASTPreparedParameter &>();
return {nullptr, DataTypeFactory::instance().get(prepared_param.type), node->getColumnName()};
}
ColumnWithTypeAndName ExprAnalyzerVisitor::analyzeOrdinaryFunction(ASTFunctionPtr & function)
{
auto overload_resolver = FunctionFactory::instance().get(function->name, context);

View File

@ -45,6 +45,10 @@ protected:
virtual R visitExistsSubquery(ASTPtr & node, ASTFunction & ast, C & visitor_context) { return visitFunction(node, ast, visitor_context); }
virtual R visitScalarSubquery(ASTPtr & node, ASTSubquery & ast, C & visitor_context) { return visitExpression(node, ast, visitor_context); }
virtual R visitQuantifiedComparisonSubquery(ASTPtr & node, ASTQuantifiedComparison & ast, C & visitor_context) { return visitExpression(node, ast, visitor_context); }
virtual R visitPreparedParameter(ASTPtr & node, ASTPreparedParameter & ast, C & visitor_context)
{
return visitExpression(node, ast, visitor_context);
}
public:
explicit AnalyzerExpressionVisitor(ContextPtr context_): context(context_) {}
@ -104,6 +108,11 @@ public:
{
return visitQuantifiedComparisonSubquery(node, node->as<ASTQuantifiedComparison &>(), visitor_context);
}
R visitASTPreparedParameter(ASTPtr & node, C & visitor_context) final
{
return visitPreparedParameter(node, node->as<ASTPreparedParameter &>(), visitor_context);
}
};
/// ExpressionTreeVisitor is used to provide default traversal logic for expression asts. (used for analyzed expr)
@ -176,7 +185,8 @@ public:
void process(ASTPtr & node, const Void & traversal_context) override
{
// node is expression AST
if (node->as<ASTIdentifier>() || node->as<ASTFunction>() || node->as<ASTLiteral>() || node->as<ASTSubquery>() || node->as<ASTFieldReference>() || node->as<ASTQuantifiedComparison>())
if (node->as<ASTIdentifier>() || node->as<ASTFunction>() || node->as<ASTLiteral>() || node->as<ASTSubquery>()
|| node->as<ASTFieldReference>() || node->as<ASTQuantifiedComparison>() || node->as<ASTPreparedParameter>())
user_visitor.process(node, user_context);
return ASTVisitorUtil::accept(node, *this, traversal_context);

View File

@ -13,8 +13,6 @@
* limitations under the License.
*/
#include <sstream>
#include <unordered_map>
#include <Analyzers/ExprAnalyzer.h>
#include <Analyzers/ExpressionVisitor.h>
#include <Analyzers/QueryAnalyzer.h>
@ -38,6 +36,7 @@
#include <Parsers/ASTColumnsMatcher.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTFieldReference.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
@ -45,8 +44,10 @@
#include <Parsers/queryToString.h>
#include <QueryPlan/Void.h>
#include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/StorageDistributed.h>
#include <Storages/StorageMemory.h>
#if USE_HIVE
# include <Storages/Hive/StorageCnchHive.h>
#endif
@ -99,6 +100,14 @@ public:
Void visitASTSelectQuery(ASTPtr & node, const Void &) override;
Void visitASTSubquery(ASTPtr & node, const Void &) override;
Void visitASTExplainQuery(ASTPtr & node, const Void &) override;
Void visitASTCreatePreparedStatementQuery(ASTPtr & node, const Void &) override
{
auto & prepare = node->as<ASTCreatePreparedStatementQuery &>();
auto query = prepare.getQuery();
process(query);
analysis.setOutputDescription(*node, analysis.getOutputDescription(*query));
return {};
}
QueryAnalyzerVisitor(ContextPtr context_, Analysis & analysis_, ScopePtr outer_query_scope_)
: context(std::move(context_))

View File

@ -0,0 +1,137 @@
#include <Analyzers/SubstituteLiteralToPreparedParams.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Parsers/ASTPreparedParameter.h>
#include <Common/FieldVisitors.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
};
namespace
{
class FieldToDataTypeRelaxed : public StaticVisitor<DataTypePtr>
{
public:
DataTypePtr operator()(const Null &) const
{
return nullptr;
}
DataTypePtr operator()(const NegativeInfinity &) const
{
return nullptr;
}
DataTypePtr operator()(const PositiveInfinity &) const
{
return nullptr;
}
DataTypePtr operator()(const UInt64 &) const
{
return std::make_shared<DataTypeUInt64>();
}
DataTypePtr operator()(const UInt128 &) const
{
return nullptr;
}
DataTypePtr operator()(const Int64 &) const
{
return std::make_shared<DataTypeInt64>();
}
DataTypePtr operator()(const Int128 &) const
{
return nullptr;
}
DataTypePtr operator()(const UUID &) const
{
return nullptr;
}
DataTypePtr operator()(const IPv4 &) const
{
return nullptr;
}
DataTypePtr operator()(const IPv6 &) const
{
return nullptr;
}
DataTypePtr operator()(const Float64 &) const
{
return nullptr;
}
DataTypePtr operator()(const String &) const
{
return std::make_shared<DataTypeString>();
}
DataTypePtr operator()(const Array &) const
{
return nullptr;
}
DataTypePtr operator()(const Tuple &) const
{
return nullptr;
}
DataTypePtr operator()(const Map &) const
{
return nullptr;
}
DataTypePtr operator()(const Object &) const
{
return nullptr;
}
DataTypePtr operator()(const DecimalField<Decimal32> &) const
{
return nullptr;
}
DataTypePtr operator()(const DecimalField<Decimal64> &) const
{
return nullptr;
}
DataTypePtr operator()(const DecimalField<Decimal128> &) const
{
return nullptr;
}
DataTypePtr operator()(const DecimalField<Decimal256> &) const
{
return nullptr;
}
DataTypePtr operator()(const AggregateFunctionStateData &) const
{
return nullptr;
}
DataTypePtr operator()(const BitMap64 &) const
{
return nullptr;
}
DataTypePtr operator()(const UInt256 &) const
{
return nullptr;
}
DataTypePtr operator()(const Int256 &) const
{
return nullptr;
}
#ifdef HAVE_BOOL_TYPE
DataTypePtr operator()(const bool &) const
{
return nullptr;
}
#endif
};
};
void SubstituteLiteralToPreparedParamsMatcher::visit(ASTLiteral & literal, ASTPtr & ast, Data & data)
{
DataTypePtr param_type = applyVisitor(FieldToDataTypeRelaxed(), literal.value);
if (!param_type)
return;
String param_name = "p" + std::to_string(data.next_param_id++);
data.extracted_binding.emplace(PreparedParameter{param_name, param_type}, literal.value);
ast = std::make_shared<ASTPreparedParameter>(param_name, param_type->getName());
}
}

View File

@ -0,0 +1,42 @@
#pragma once
#include <Interpreters/Context.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/prepared_statement.h>
#include <Parsers/ASTLiteral.h>
namespace DB
{
class SubstituteLiteralToPreparedParamsMatcher
{
public:
class Data
{
public:
explicit Data(ContextPtr context_) : context(std::move(context_))
{
}
ContextPtr context;
PreparedParameterBindings extracted_binding;
int next_param_id = 0;
};
static bool needChildVisit(ASTPtr &, const ASTPtr &)
{
return true;
}
static void visit(ASTPtr & ast, Data & data)
{
if (auto * literal = ast->as<ASTLiteral>())
visit(*literal, ast, data);
}
static void visit(ASTLiteral & literal, ASTPtr & ast, Data & data);
};
using SubstituteLiteralToPreparedParamsVisitor = InDepthNodeVisitor<SubstituteLiteralToPreparedParamsMatcher, true>;
}

View File

@ -240,6 +240,8 @@ add_object_library(clickhouse_databases_mysql Databases/MySQL)
add_object_library(clickhouse_disks Disks)
add_object_library(clickhouse_interpreters Interpreters)
add_object_library(clickhouse_interpreters_cache Interpreters/Cache)
add_object_library(clickhouse_interpreters_binding Interpreters/SQLBinding)
add_object_library(clickhouse_interpreters_preparedstatement Interpreters/PreparedStatement)
add_object_library(clickhouse_interpreters_mysql Interpreters/MySQL)
add_object_library(clickhouse_interpreters_clusterproxy Interpreters/ClusterProxy)
add_object_library(clickhouse_interpreters_profile Interpreters/profile)

View File

@ -902,9 +902,13 @@
\
M(11013, GLUE_CATALOG_RPC_ERROR) \
\
M(12000, INCORRECT_UDF_FUNCTION_NAME)
/* See END */
M(12000, INCORRECT_UDF_FUNCTION_NAME)\
M(4041, UNKNOWN_BINDING) \
M(4042, PREPARED_STATEMENT_NOT_EXISTS) \
M(4043, PREPARED_STATEMENT_ALREADY_EXISTS) \
M(4080, BAD_PREPARED_PARAMETER) \
M(4081, PLAN_CACHE_NOT_USED) \
/* See END */
namespace DB
{

View File

@ -22,6 +22,7 @@
#include <Common/SettingsChanges.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Protos/plan_node_utils.pb.h>
namespace DB
{
@ -56,6 +57,18 @@ void SettingChange::deserialize(ReadBuffer & buf)
readFieldBinary(value, buf);
}
void SettingChange::toProto(Protos::SettingChange & proto) const
{
proto.set_name(name);
value.toProto(*proto.mutable_value());
}
void SettingChange::fillFromProto(const Protos::SettingChange & proto)
{
name = proto.name();
value.fillFromProto(proto.value());
}
bool SettingsChanges::tryGet(const std::string_view & name, Field & out_value) const
{
const auto * change = find(*this, name);
@ -81,6 +94,38 @@ Field * SettingsChanges::tryGet(const std::string_view & name)
return &change->value;
}
bool SettingsChanges::insertSetting(std::string_view name, const Field & value)
{
auto it = std::find_if(begin(), end(), [&name](const SettingChange & change) { return change.name == name; });
if (it != end())
return false;
emplace_back(name, value);
return true;
}
void SettingsChanges::setSetting(std::string_view name, const Field & value)
{
if (auto * setting_value = tryGet(name))
*setting_value = value;
else
insertSetting(name, value);
}
bool SettingsChanges::removeSetting(std::string_view name)
{
auto it = std::find_if(begin(), end(), [&name](const SettingChange & change) { return change.name == name; });
if (it == end())
return false;
erase(it);
return true;
}
void SettingsChanges::merge(const SettingsChanges & other)
{
for (const auto & change : other)
setSetting(change.name, change.value);
}
void SettingsChanges::serialize(WriteBuffer & buf) const
{
writeBinary(size(), buf);
@ -100,4 +145,20 @@ void SettingsChanges::deserialize(ReadBuffer & buf)
}
}
void SettingsChanges::toProto(Protos::SettingsChanges & proto) const
{
for (const auto & element : *this)
element.toProto(*proto.add_settings_changes());
}
void SettingsChanges::fillFromProto(const Protos::SettingsChanges & proto)
{
for (const auto & proto_element : proto.settings_changes())
{
SettingChange element;
element.fillFromProto(proto_element);
this->emplace_back(std::move(element));
}
}
}

View File

@ -26,6 +26,11 @@
namespace DB
{
namespace Protos
{
class SettingChange;
class SettingsChanges;
}
struct SettingChange
{
String name;
@ -40,6 +45,8 @@ struct SettingChange
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
void toProto(Protos::SettingChange & proto) const;
void fillFromProto(const Protos::SettingChange & proto);
};
@ -52,8 +59,19 @@ public:
const Field * tryGet(const std::string_view & name) const;
Field * tryGet(const std::string_view & name);
/// Inserts element if doesn't exists and returns true, otherwise just returns false
bool insertSetting(std::string_view name, const Field & value);
/// Sets element to value, inserts if doesn't exist
void setSetting(std::string_view name, const Field & value);
/// If element exists - removes it and returns true, otherwise returns false
bool removeSetting(std::string_view name);
void merge(const SettingsChanges & other);
void serialize(WriteBuffer & buf) const;
void deserialize(ReadBuffer & buf);
void toProto(Protos::SettingsChanges & proto) const;
void fillFromProto(const Protos::SettingsChanges & proto);
};
}

223
src/Common/atomicRename.cpp Normal file
View File

@ -0,0 +1,223 @@
#include <Common/atomicRename.h>
#include <Common/Exception.h>
#include <Common/VersionNumber.h>
#include <Poco/Environment.h>
#include <filesystem>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ATOMIC_RENAME_FAIL;
extern const int SYSTEM_ERROR;
extern const int UNSUPPORTED_METHOD;
extern const int FILE_ALREADY_EXISTS;
}
}
#if defined(OS_LINUX)
#include <unistd.h>
#include <fcntl.h>
#include <sys/syscall.h>
#include <linux/fs.h>
/// For old versions of libc.
#if !defined(RENAME_NOREPLACE)
#define RENAME_NOREPLACE 1
#endif
#if !defined(RENAME_EXCHANGE)
#define RENAME_EXCHANGE 2
#endif
#if !defined(__NR_renameat2)
#if defined(__x86_64__)
#define __NR_renameat2 316
#elif defined(__aarch64__)
#define __NR_renameat2 276
#elif defined(__ppc64__)
#define __NR_renameat2 357
#elif defined(__riscv)
#define __NR_renameat2 276
#else
#error "Unsupported architecture"
#endif
#endif
namespace DB
{
static bool supportsAtomicRenameImpl()
{
VersionNumber renameat2_minimal_version(3, 15, 0);
VersionNumber linux_version(Poco::Environment::osVersion());
return linux_version >= renameat2_minimal_version;
}
static bool renameat2(const std::string & old_path, const std::string & new_path, int flags)
{
if (!supportsAtomicRename())
return false;
if (old_path.empty() || new_path.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path);
/// int olddirfd (ignored for absolute oldpath), const char *oldpath,
/// int newdirfd (ignored for absolute newpath), const char *newpath,
/// unsigned int flags
if (0 == syscall(__NR_renameat2, AT_FDCWD, old_path.c_str(), AT_FDCWD, new_path.c_str(), flags))
return true;
/// EINVAL means that filesystem does not support one of the flags.
/// It also may happen when running clickhouse in docker with Mac OS as a host OS.
/// supportsRenameat2() with uname is not enough in this case, because virtualized Linux kernel is used.
/// Other cases when EINVAL can be returned should never happen.
if (errno == EINVAL)
return false;
/// We should never get ENOSYS on Linux, because we check kernel version in supportsRenameat2Impl().
/// However, we can get in on WSL.
if (errno == ENOSYS)
return false;
if (errno == EEXIST)
throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
if (errno == ENOENT)
throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
throwFromErrnoWithPath(fmt::format("Cannot rename {} to {}", old_path, new_path), new_path, ErrorCodes::SYSTEM_ERROR);
}
bool supportsAtomicRename()
{
static bool supports = supportsAtomicRenameImpl();
return supports;
}
}
#elif defined(OS_DARWIN)
// Includes
#include <dlfcn.h> // For dlsym
#include <stdio.h> // For renamex_np
#include <string.h> // For stderror
#ifndef RENAME_SWAP
#define RENAME_SWAP 0x00000002
#endif
#ifndef RENAME_EXCL
#define RENAME_EXCL 0x00000004
#endif
#define RENAME_NOREPLACE RENAME_EXCL
#define RENAME_EXCHANGE RENAME_SWAP
namespace DB
{
static bool renameat2(const std::string & old_path, const std::string & new_path, int flags)
{
using function_type = int (*)(const char * from, const char * to, unsigned int flags);
static function_type fun = reinterpret_cast<function_type>(dlsym(RTLD_DEFAULT, "renamex_np"));
if (fun == nullptr)
return false;
if (old_path.empty() || new_path.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot rename {} to {}: path is empty", old_path, new_path);
if (0 == (*fun)(old_path.c_str(), new_path.c_str(), flags))
return true;
int errnum = errno;
if (errnum == ENOTSUP || errnum == EINVAL)
return false;
if (errnum == EEXIST)
throwFromErrno(fmt::format("Cannot rename {} to {} because the second path already exists", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
if (errnum == ENOENT)
throwFromErrno(fmt::format("Paths cannot be exchanged because {} or {} does not exist", old_path, new_path), ErrorCodes::ATOMIC_RENAME_FAIL);
throwFromErrnoWithPath(
fmt::format("Cannot rename {} to {}: {}", old_path, new_path, strerror(errnum)), new_path, ErrorCodes::SYSTEM_ERROR);
}
static bool supportsAtomicRenameImpl()
{
auto fun = dlsym(RTLD_DEFAULT, "renamex_np");
return fun != nullptr;
}
bool supportsAtomicRename()
{
static bool supports = supportsAtomicRenameImpl();
return supports;
}
}
#else
#define RENAME_NOREPLACE -1
#define RENAME_EXCHANGE -1
namespace DB
{
static bool renameat2(const std::string &, const std::string &, int)
{
return false;
}
bool supportsAtomicRename()
{
return false;
}
}
#endif
namespace DB
{
static void renameNoReplaceFallback(const std::string & old_path, const std::string & new_path)
{
/// NOTE it's unsafe
if (fs::exists(new_path))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "File {} exists", new_path);
fs::rename(old_path, new_path);
}
/// Do not use [[noreturn]] to avoid warnings like "code will never be executed" in other places
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmissing-noreturn"
static void renameExchangeFallback(const std::string &, const std::string &)
{
throw Exception("System call renameat2() is not supported", ErrorCodes::UNSUPPORTED_METHOD);
}
#pragma GCC diagnostic pop
void renameNoReplace(const std::string & old_path, const std::string & new_path)
{
if (!renameat2(old_path, new_path, RENAME_NOREPLACE))
renameNoReplaceFallback(old_path, new_path);
}
void renameExchange(const std::string & old_path, const std::string & new_path)
{
if (!renameat2(old_path, new_path, RENAME_EXCHANGE))
renameExchangeFallback(old_path, new_path);
}
bool renameExchangeIfSupported(const std::string & old_path, const std::string & new_path)
{
return renameat2(old_path, new_path, RENAME_EXCHANGE);
}
}

20
src/Common/atomicRename.h Normal file
View File

@ -0,0 +1,20 @@
#pragma once
#include <string>
namespace DB
{
/// Returns true, if the following functions supported by the system
bool supportsAtomicRename();
/// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception
void renameNoReplace(const std::string & old_path, const std::string & new_path);
/// Atomically exchange oldpath and newpath. Throw exception if some of them does not exist
void renameExchange(const std::string & old_path, const std::string & new_path);
/// Returns false instead of throwing exception if renameat2 is not supported
bool renameExchangeIfSupported(const std::string & old_path, const std::string & new_path);
}

View File

@ -63,6 +63,7 @@ template <class Traits_>
class BaseSettings : public Traits_::Data
{
using CustomSettingMap = std::unordered_map<std::string_view, std::pair<std::shared_ptr<const String>, SettingFieldCustom>>;
using Whitelist = std::unordered_set<String>;
public:
using Traits = Traits_;
@ -101,7 +102,7 @@ public:
static String valueToStringUtil(const std::string_view & name, const Field & value);
static Field stringToValueUtil(const std::string_view & name, const String & str);
void write(WriteBuffer & out, SettingsWriteFormat write_format = SettingsWriteFormat::DEFAULT) const;
void write(WriteBuffer & out, SettingsWriteFormat write_format = SettingsWriteFormat::DEFAULT, Whitelist white_list = {}) const;
void read(ReadBuffer & in, SettingsWriteFormat read_format = SettingsWriteFormat::DEFAULT);
// A debugging aid.
@ -431,7 +432,7 @@ Field BaseSettings<Traits_>::stringToValueUtil(const std::string_view & name, co
}
template <typename Traits_>
void BaseSettings<Traits_>::write(WriteBuffer & out, SettingsWriteFormat write_format) const
void BaseSettings<Traits_>::write(WriteBuffer & out, SettingsWriteFormat write_format, Whitelist white_list) const
{
const auto & accessor = Traits::Accessor::instance();
@ -440,6 +441,10 @@ void BaseSettings<Traits_>::write(WriteBuffer & out, SettingsWriteFormat write_f
bool is_custom = field.isCustom();
bool is_important = !is_custom && accessor.isImportant(field.index);
// Skip serialization of settings in the whitelist
if (white_list.count(field.getName()))
continue;
BaseSettingsHelpers::writeString(field.getName(), out);
if ((write_format >= SettingsWriteFormat::STRINGS_WITH_FLAGS) || is_custom)

View File

@ -1533,6 +1533,7 @@ enum PreloadLevelSettings : UInt64
M(UInt64, plan_optimizer_timeout, 600000, "Max running time of a plan rewriter optimizer in ms", 0) \
M(UInt64, plan_optimizer_rule_warning_time, 1000, "Send warning if a optimize rule optimize time exceed timeout", 0) \
M(Bool, enable_plan_cache, false, "Whether enable plan cache", 0) \
M(Bool, force_plan_cache, false, "Force to use plan cache", 0) \
M(UInt64, max_plannode_count, 200, "The max plannode count", 0) \
M(Bool, enable_memory_catalog, false, "Enable memory catalog for unittest", 0) \
M(UInt64, memory_catalog_worker_size, 8, "Memory catalog work size for unittest", 0) \
@ -1548,6 +1549,7 @@ enum PreloadLevelSettings : UInt64
/** */ \
M(Bool, late_materialize_aggressive_push_down, false, "When table use early materialize strategy, this setting enable aggressively moving predicates to read chain w/o considering other factor like columns size or number of columns in the query", 0) \
/** Optimizer relative settings, Plan build and RBO */ \
M(Bool, enable_auto_prepared_statement, false, "Whether to enable automatic prepared statement", 0) \
M(Bool, enable_nested_loop_join, true, "Whether enable nest loop join for outer join with filter", 0)\
M(Bool, enforce_all_join_to_any_join, false, "Whether enforce all join to any join", 0) \
M(Bool, enable_implicit_type_conversion, true, "Whether enable implicit type conversion for JOIN, Set operation, IN subquery", 0) \

View File

@ -25,7 +25,7 @@
#include <IO/WriteHelpers.h>
#include <IO/ReadBufferFromFile.h>
#include <Parsers/formatAST.h>
#include <Common/renameat2.h>
#include <Common/atomicRename.h>
#include <Storages/StorageMaterializedView.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
@ -178,7 +178,7 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
return;
}
if (exchange && !supportsRenameat2())
if (exchange && !supportsAtomicRename())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported");
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);

View File

@ -68,6 +68,8 @@ const char * ActionsDAG::typeToString(ActionsDAG::ActionType type)
return "ArrayJoin";
case ActionType::FUNCTION:
return "Function";
case ActionType::PREPARED_COLUMN:
return "PREPARED_COLUMN";
}
__builtin_unreachable();
@ -394,6 +396,16 @@ const ActionsDAG::Node & ActionsDAG::addFunction(
return addNode(std::move(node));
}
const ActionsDAG::Node & ActionsDAG::addPreparedColumn(std::string name, DataTypePtr type)
{
Node node;
node.type = ActionType::PREPARED_COLUMN;
node.result_type = std::move(type);
node.result_name = std::move(name);
return addNode(std::move(node));
}
const ActionsDAG::Node & ActionsDAG::findInOutputs(const std::string & name) const
{
if (const auto * node = tryFindInOutputs(name))
@ -654,6 +666,11 @@ static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * nod
{
break;
}
case ActionsDAG::ActionType::PREPARED_COLUMN:
{
break;
}
}
return res_column;
@ -1089,6 +1106,10 @@ std::string ActionsDAG::dumpDAG() const
case ActionsDAG::ActionType::INPUT:
out << "INPUT ";
break;
case ActionsDAG::ActionType::PREPARED_COLUMN:
out << "PREPARED_COLUMN ";
break;
}
out << "(";

View File

@ -71,7 +71,6 @@ namespace JSONBuilder
class ActionsDAG
{
public:
enum class ActionType
{
/// Column which must be in input.
@ -83,6 +82,8 @@ public:
/// Function arrayJoin. Specially separated because it changes the number of rows.
ARRAY_JOIN,
FUNCTION,
/// A constant column which is not determined yet
PREPARED_COLUMN,
};
static const char * typeToString(ActionType type);
@ -176,6 +177,7 @@ public:
const FunctionOverloadResolverPtr & function,
NodeRawConstPtrs children,
std::string result_name);
const Node & addPreparedColumn(std::string name, DataTypePtr type);
/// Find first column by name in output nodes. This search is linear.
const Node & findInOutputs(const std::string & name) const;

View File

@ -29,13 +29,19 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeSet.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeBitMap64.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeFunction.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeSet.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/FieldToDataType.h>
#include <Columns/ColumnSet.h>
@ -725,6 +731,8 @@ void ActionsMatcher::visit(const ASTPtr & ast, Data & data)
visit(*literal, ast, data);
else if (auto * expression_list = ast->as<ASTExpressionList>())
visit(*expression_list, ast, data);
else if (auto * prepared_param = ast->as<ASTPreparedParameter>())
visit(*prepared_param, ast, data);
else
{
for (auto & child : ast->children)
@ -857,6 +865,19 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Dat
}
}
void ActionsMatcher::visit(const ASTPreparedParameter & prepared_param, const ASTPtr &, Data & data)
{
auto column_name = prepared_param.getColumnName();
if (data.hasColumn(column_name))
return;
auto column_type = DataTypeFactory::instance().get(prepared_param.type);
auto & level = data.actions_stack.stack.back();
const auto & node = level.actions_dag->addPreparedColumn(column_name, column_type);
level.index->addNode(&node);
}
static bool checkIdentifier(const ASTPtr & node)
{
if (!node)

View File

@ -31,6 +31,9 @@
#include <Parsers/ASTExpressionList.h>
#include <Storages/MergeTree/Index/BitmapIndexHelper.h>
#include <Storages/MergeTree/Index/MergeTreeIndexHelper.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTPreparedParameter.h>
#include <Parsers/IAST.h>
#include <Storages/StorageInMemoryMetadata.h>
namespace DB
@ -275,6 +278,7 @@ private:
static void visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data);
static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data);
static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data);
static void visit(const ASTPreparedParameter & prepared_param, const ASTPtr & ast, Data & data);
static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data);
static SetPtr tryMakeSet(const ASTFunction & node, Data & data, bool no_subqueries, bool create_ordered_set = false);

View File

@ -31,6 +31,7 @@
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTWithElement.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/DumpASTNode.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
@ -48,8 +49,17 @@ namespace DB
class AddDefaultDatabaseVisitor
{
public:
struct Data
{
std::set<String> subqueries;
};
explicit AddDefaultDatabaseVisitor(
ContextPtr context_, const String & database_name_, bool only_replace_current_database_function_ = false, bool only_replace_in_join_ = false, WriteBuffer * ostr_ = nullptr)
ContextPtr context_,
const String & database_name_,
bool only_replace_current_database_function_ = false,
bool only_replace_in_join_ = false,
WriteBuffer * ostr_ = nullptr)
: context(context_)
, database_name(database_name_)
, only_replace_current_database_function(only_replace_current_database_function_)
@ -78,22 +88,19 @@ public:
void visit(ASTPtr & ast) const
{
if (!tryVisit<ASTSelectQuery>(ast) &&
!tryVisit<ASTSelectWithUnionQuery>(ast) &&
!tryVisit<ASTFunction>(ast))
visitChildren(*ast);
Data data;
visit(ast, data);
}
void visit(ASTSelectQuery & select) const
{
ASTPtr unused;
visit(select, unused);
Data data;
visit(select, data);
}
void visit(ASTSelectWithUnionQuery & select) const
{
ASTPtr unused;
visit(select, unused);
Data data;
visit(select, data);
}
private:
@ -107,76 +114,112 @@ private:
mutable size_t visit_depth;
WriteBuffer * ostr;
void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const
void visit(ASTPtr & ast, Data & data) const
{
if (!tryVisit<ASTSelectQuery>(ast, data) && !tryVisit<ASTSelectWithUnionQuery>(ast, data) && !tryVisit<ASTFunction>(ast, data))
visitChildren(*ast, data);
}
void visit(ASTSelectQuery & select, Data & data) const
{
ASTPtr unused;
visit(select, unused, data);
}
void visit(ASTSelectWithUnionQuery & select, Data & data) const
{
ASTPtr unused;
visit(select, unused, data);
}
void visit(ASTSelectWithUnionQuery & select, ASTPtr &, Data & data) const
{
for (auto & child : select.list_of_selects->children)
{
if (child->as<ASTSelectQuery>())
tryVisit<ASTSelectQuery>(child);
tryVisit<ASTSelectQuery>(child, data);
else if (child->as<ASTSelectIntersectExceptQuery>())
tryVisit<ASTSelectIntersectExceptQuery>(child);
tryVisit<ASTSelectIntersectExceptQuery>(child, data);
}
}
void visit(ASTSelectQuery & select, ASTPtr &) const
void visit(ASTSelectQuery & select, ASTPtr &, Data & data) const
{
if (select.tables())
tryVisit<ASTTablesInSelectQuery>(select.refTables());
std::optional<Data> new_data;
if (auto with = select.with())
{
for (auto & child : with->children)
{
visit(child, new_data ? *new_data : data);
if (auto * ast_with_elem = child->as<ASTWithElement>())
{
if (!new_data)
new_data = data;
new_data->subqueries.emplace(ast_with_elem->name);
}
}
}
visitChildren(select);
if (select.tables())
tryVisit<ASTTablesInSelectQuery>(select.refTables(), new_data ? *new_data : data);
visitChildren(select, new_data ? *new_data : data);
}
void visit(ASTSelectIntersectExceptQuery & select, ASTPtr &) const
void visit(ASTSelectIntersectExceptQuery & select, ASTPtr &, Data & data) const
{
for (auto & child : select.getListOfSelects())
{
if (child->as<ASTSelectQuery>())
tryVisit<ASTSelectQuery>(child);
tryVisit<ASTSelectQuery>(child, data);
else if (child->as<ASTSelectIntersectExceptQuery>())
tryVisit<ASTSelectIntersectExceptQuery>(child);
tryVisit<ASTSelectIntersectExceptQuery>(child, data);
else if (child->as<ASTSelectWithUnionQuery>())
tryVisit<ASTSelectWithUnionQuery>(child);
tryVisit<ASTSelectWithUnionQuery>(child, data);
}
}
void visit(ASTTablesInSelectQuery & tables, ASTPtr &) const
void visit(ASTTablesInSelectQuery & tables, ASTPtr &, Data & data) const
{
for (auto & child : tables.children)
tryVisit<ASTTablesInSelectQueryElement>(child);
tryVisit<ASTTablesInSelectQueryElement>(child, data);
}
void visit(ASTTablesInSelectQueryElement & tables_element, ASTPtr &) const
void visit(ASTTablesInSelectQueryElement & tables_element, ASTPtr &, Data & data) const
{
if (only_replace_in_join && !tables_element.table_join)
return;
if (tables_element.table_expression)
tryVisit<ASTTableExpression>(tables_element.table_expression);
tryVisit<ASTTableExpression>(tables_element.table_expression, data);
}
void visit(ASTTableExpression & table_expression, ASTPtr &) const
void visit(ASTTableExpression & table_expression, ASTPtr &, Data & data) const
{
if (table_expression.database_and_table_name)
tryVisit<ASTTableIdentifier>(table_expression.database_and_table_name);
tryVisit<ASTTableIdentifier>(table_expression.database_and_table_name, data);
}
void visit(const ASTTableIdentifier & identifier, ASTPtr & ast) const
void visit(const ASTTableIdentifier & identifier, ASTPtr & ast, Data & data) const
{
/// Already has database.
if (identifier.compound())
return;
/// There is temporary table with such name, should not be rewritten.
/// There is temporary table with such name, should not be rewritten.
if (external_tables.count(identifier.shortName()))
return;
if (data.subqueries.count(identifier.shortName()))
return;
auto qualified_identifier = std::make_shared<ASTTableIdentifier>(database_name, identifier.name());
if (!identifier.alias.empty())
qualified_identifier->setAlias(identifier.alias);
ast = qualified_identifier;
}
void visit(ASTFunction & function, ASTPtr &) const
void visit(ASTFunction & function, ASTPtr &, Data & data) const
{
bool is_operator_in = false;
for (const auto * name : {"in", "notIn", "globalIn", "globalNotIn"})
@ -203,31 +246,31 @@ private:
/// Second argument of the "in" function (or similar) may be a table name or a subselect.
/// Rewrite the table name or descend into subselect.
if (!tryVisit<ASTTableIdentifier>(child->children[i]))
visit(child->children[i]);
if (!tryVisit<ASTTableIdentifier>(child->children[i], data))
visit(child->children[i], data);
}
else
visit(child->children[i]);
visit(child->children[i], data);
}
}
else
visit(child);
visit(child, data);
}
}
void visitChildren(IAST & ast) const
void visitChildren(IAST & ast, Data & data) const
{
for (auto & child : ast.children)
visit(child);
visit(child, data);
}
template <typename T>
bool tryVisit(ASTPtr & ast) const
bool tryVisit(ASTPtr & ast, Data & data) const
{
if (T * t = typeid_cast<T *>(ast.get()))
{
DumpASTNode dump(*ast, ostr, visit_depth, "addDefaultDatabaseName");
visit(*t, ast);
visit(*t, ast, data);
return true;
}
return false;

View File

@ -35,9 +35,9 @@
#include <Access/ExternalAuthenticators.h>
#include <Access/GSSAcceptor.h>
#include <Access/QuotaUsage.h>
#include <Access/SettingsProfilesInfo.h>
#include <Access/SettingsConstraintsAndProfileIDs.h>
#include <Access/SettingsProfile.h>
#include <Access/SettingsProfilesInfo.h>
#include <Access/User.h>
#include <Catalog/Catalog.h>
#include <CloudServices/CnchBGThreadsMap.h>
@ -85,9 +85,12 @@
#include <Interpreters/InterserverIOHandler.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
#include <Interpreters/NamedSession.h>
#include <Interpreters/Lemmatizers.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/QueueManager.h>
#include <Interpreters/SegmentScheduler.h>
#include <Interpreters/SynonymsExtensions.h>
#include <Interpreters/SystemLog.h>
#include <Interpreters/VirtualWarehousePool.h>
#include <Interpreters/WorkerGroupHandle.h>
@ -474,6 +477,9 @@ struct ContextSharedPart
std::unique_ptr<Statistics::AutoStats::AutoStatisticsManager> auto_stats_manager;
std::unique_ptr<PlanCacheManager> plan_cache_manager;
std::unique_ptr<PreparedStatementManager> prepared_statement_manager;
ContextSharedPart()
: macros(std::make_unique<Macros>())
{
@ -585,6 +591,8 @@ struct ContextSharedPart
///
/// But they cannot be created before storages since they may required table as a source,
/// but at least they can be preserved for storage termination.
prepared_statement_manager.reset();
dictionaries_xmls.reset();
dictionaries_cnch_catalog.reset();
@ -5722,6 +5730,18 @@ PlanCacheManager* Context::getPlanCacheManager()
return shared->plan_cache_manager ? shared->plan_cache_manager.get() : nullptr;
}
void Context::setPreparedStatementManager(std::unique_ptr<PreparedStatementManager> && manager)
{
auto lock = getLock();
shared->prepared_statement_manager = std::move(manager);
}
PreparedStatementManager * Context::getPreparedStatementManager()
{
auto lock = getLock();
return shared->prepared_statement_manager ? shared->prepared_statement_manager.get() : nullptr;
}
UInt32 Context::getQueryMaxExecutionTime() const
{
// max is 4294967295/1000/60=71582 min

View File

@ -356,6 +356,7 @@ using ExcludedRules = std::unordered_set<UInt32>;
using ExcludedRulesMap = std::unordered_map<PlanNodeId, ExcludedRules>;
class PlanCacheManager;
class PreparedStatementManager;
/// An empty interface for an arbitrary object that may be attached by a shared pointer
/// to query context, when using ClickHouse as a library.
@ -1627,6 +1628,9 @@ public:
AsynchronousReaderPtr getThreadPoolReader() const;
void setPreparedStatementManager(std::unique_ptr<PreparedStatementManager> && manager);
PreparedStatementManager * getPreparedStatementManager();
private:
String tenant_id;
String current_catalog;

View File

@ -33,7 +33,7 @@
#include <Parsers/formatAST.h>
#include <IO/ReadHelpers.h>
#include <Poco/DirectoryIterator.h>
#include <Common/renameat2.h>
#include <Common/atomicRename.h>
#include <Common/CurrentMetrics.h>
#include <common/logger_useful.h>
#include <Poco/Util/AbstractConfiguration.h>

View File

@ -493,6 +493,10 @@ std::string ExpressionActions::Action::toString() const
case ActionsDAG::ActionType::INPUT:
out << "INPUT " << arguments.front();
break;
case ActionsDAG::ActionType::PREPARED_COLUMN:
out << "PREPARED_COLUMN " << arguments.front();
break;
}
out << " -> " << node->result_name
@ -667,6 +671,14 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon
break;
}
case ActionsDAG::ActionType::PREPARED_COLUMN: {
auto & res_column = columns[action.result_position];
res_column.column = action.node->result_type->createColumn();
res_column.type = action.node->result_type;
res_column.name = action.node->result_name;
break;
}
case ActionsDAG::ActionType::ALIAS:
{
const auto & arg = action.arguments.front();

View File

@ -29,7 +29,7 @@
#include <Common/typeid_cast.h>
#include <Common/Macros.h>
#include <Common/randomSeed.h>
#include <Common/renameat2.h>
#include <Common/atomicRename.h>
#include <Core/Defines.h>
#include <Core/Settings.h>

View File

@ -0,0 +1,34 @@
#include <DataStreams/OneBlockInputStream.h>
#include <Interpreters/InterpreterDropPreparedStatementQuery.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Parsers/ASTPreparedStatement.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_PREPARE;
}
BlockIO InterpreterDropPreparedStatementQuery::execute()
{
const auto * drop = query_ptr->as<const ASTDropPreparedStatementQuery>();
if (!drop || drop->name.empty())
throw Exception("Drop Prepare logical error", ErrorCodes::LOGICAL_ERROR);
auto current_context = getContext();
// if (!drop->cluster.empty())
// return executeDDLQueryOnCluster(query_ptr, current_context);
auto * prepared_manager = current_context->getPreparedStatementManager();
// get prepare from cache
if (!prepared_manager)
throw Exception("Prepare cache has to be initialized", ErrorCodes::LOGICAL_ERROR);
prepared_manager->remove(drop->name, !drop->if_exists);
return {};
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
namespace DB
{
class InterpreterDropPreparedStatementQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterDropPreparedStatementQuery(const ASTPtr & query_ptr_, ContextMutablePtr & context_)
: WithMutableContext(context_), query_ptr(query_ptr_)
{
}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -689,8 +689,8 @@ void InterpreterExplainQuery::explainUsingOptimizer(const ASTPtr & ast, WriteBuf
}
InterpreterSelectQueryUseOptimizer interpreter(explain.getExplainedQuery(), context, SelectQueryOptions());
auto query_plan = interpreter.buildQueryPlan();
if (explain.getKind() == ASTExplainQuery::ExplainKind::OptimizerPlan || explain.getKind() == ASTExplainQuery::ExplainKind::QueryPlan)
auto query_plan = interpreter.getQueryPlan();
if (explain.getKind() == ASTExplainQuery::ExplainKind::OptimizerPlan || explain.getKind() == ASTExplainQuery::ExplainKind::QueryPlan )
{
explainPlanWithOptimizer(explain, *query_plan, buffer, context, single_line);
}

View File

@ -43,6 +43,7 @@
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTKillQueryQuery.h>
#include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTRenameQuery.h>
#include <Parsers/ASTReproduceQuery.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
@ -92,6 +93,7 @@
#include <Interpreters/InterpreterDropAccessEntityQuery.h>
#include <Interpreters/InterpreterDropWarehouseQuery.h>
#include <Interpreters/InterpreterDropWorkerGroupQuery.h>
#include <Interpreters/InterpreterDropPreparedStatementQuery.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/InterpreterUndropQuery.h>
#include <Interpreters/InterpreterDumpQuery.h>
@ -118,6 +120,7 @@
#include <Interpreters/InterpreterShowCreateAccessEntityQuery.h>
#include <Interpreters/InterpreterShowCreateQuery.h>
#include <Interpreters/InterpreterShowGrantsQuery.h>
#include <Interpreters/InterpreterShowPreparedStatementQuery.h>
#include <Interpreters/InterpreterShowPrivilegesQuery.h>
#include <Interpreters/InterpreterShowProcesslistQuery.h>
#include <Interpreters/InterpreterShowTablesQuery.h>
@ -504,6 +507,27 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterDropBinding>(query, context);
}
else if (query->as<ASTCreatePreparedStatementQuery>())
{
if (QueryUseOptimizerChecker::check(query, context, true))
return std::make_unique<InterpreterSelectQueryUseOptimizer>(query, context, options);
throw Exception("Prepared statements requires optimizer enabled", ErrorCodes::NOT_IMPLEMENTED);
}
else if (query->as<ASTExecutePreparedStatementQuery>())
{
if (!context->getSettings().enable_optimizer)
throw Exception("Execute prepared statements requires optimizer enabled", ErrorCodes::NOT_IMPLEMENTED);
return std::make_unique<InterpreterSelectQueryUseOptimizer>(query, context, options);
}
else if (query->as<ASTShowPreparedStatementQuery>())
{
return std::make_unique<InterpreterShowPreparedStatementQuery>(query, context);
}
else if (query->as<ASTDropPreparedStatementQuery>())
{
return std::make_unique<InterpreterDropPreparedStatementQuery>(query, context);
}
else
{
throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);

View File

@ -2724,7 +2724,7 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
query_plan.getCurrentDataStream(),
w.full_sort_description,
settings.max_block_size,
0 /* LIMIT */,
size_t{0} /* LIMIT */,
settings.max_bytes_before_remerge_sort,
settings.remerge_sort_lowered_memory_bytes_ratio,
settings.max_bytes_before_external_sort,

View File

@ -12,21 +12,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <Interpreters/InterpreterSelectQueryUseOptimizer.h>
#include <Analyzers/QueryAnalyzer.h>
#include <Analyzers/QueryRewriter.h>
#include <Interpreters/Cache/QueryCache.h>
#include <Analyzers/SubstituteLiteralToPreparedParams.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/DistributedStages/MPPQueryCoordinator.h>
#include <Interpreters/DistributedStages/PlanSegment.h>
#include <Interpreters/InterpreterSelectQueryUseOptimizer.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Interpreters/SegmentScheduler.h>
#include <Interpreters/WorkerStatusManager.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Interpreters/executeQuery.h>
#include <Optimizer/JoinOrderUtils.h>
#include <Optimizer/PlanNodeSearcher.h>
#include <Optimizer/PlanOptimizer.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTTEALimit.h>
#include <Parsers/ASTWithAlias.h>
#include <Parsers/formatAST.h>
@ -49,6 +55,7 @@
#include <Common/ProfileEvents.h>
#include <common/logger_useful.h>
#include <memory>
namespace ProfileEvents
{
@ -64,22 +71,73 @@ namespace DB
namespace ErrorCodes
{
extern const int TOO_MANY_PLAN_SEGMENTS;
extern const int OPTIMIZER_NONSUPPORT;
extern const int LOGICAL_ERROR;
extern const int TOO_MANY_PLAN_SEGMENTS;
extern const int OPTIMIZER_NONSUPPORT;
extern const int LOGICAL_ERROR;
extern const int TOO_MANY_PLAN_SEGMENT;
extern const int PLAN_CACHE_NOT_USED;
extern const int BAD_PREPARED_PARAMETER;
}
Block InterpreterSelectQueryUseOptimizer::getSampleBlock()
{
if (!block)
{
auto query_plan = buildQueryPlan();
auto query_plan = getQueryPlan(true);
}
return block;
}
QueryPlanPtr InterpreterSelectQueryUseOptimizer::buildQueryPlan(bool skip_optimize)
namespace
{
struct RemoveSettings
{
using TypeToVisit = ASTSelectQuery;
void visit(ASTSelectQuery & select_query, ASTPtr &) const
{
select_query.setExpression(ASTSelectQuery::Expression::SETTINGS, nullptr);
}
};
using RemoveSettingsVisitor = InDepthNodeVisitor<OneTypeMatcher<RemoveSettings>, true>;
class CollectPreparedParams
{
public:
using TypeToVisit = ASTPreparedParameter;
PreparedParameterSet prepared_params;
void visit(ASTPreparedParameter & ast, ASTPtr &)
{
PreparedParameter param{ast.name, DataTypeFactory::instance().get(ast.type)};
auto it = prepared_params.emplace(std::move(param));
if (!it.second)
throw Exception(ErrorCodes::BAD_PREPARED_PARAMETER, "Prepared param {} is duplicated", ast.name);
}
};
using CollectPreparedParamsVisitor = InDepthNodeVisitor<OneTypeMatcher<CollectPreparedParams>, true>;
}
InterpreterSelectQueryUseOptimizer::InterpreterSelectQueryUseOptimizer(
const ASTPtr & query_ptr_,
PlanNodePtr sub_plan_ptr_,
CTEInfo cte_info_,
ContextMutablePtr & context_,
const SelectQueryOptions & options_)
: query_ptr(query_ptr_ ? query_ptr_->clone() : nullptr)
, sub_plan_ptr(sub_plan_ptr_)
, cte_info(std::move(cte_info_))
, context(context_)
, options(options_)
, log(&Poco::Logger::get("InterpreterSelectQueryUseOptimizer"))
{
interpret_sub_query = !!sub_plan_ptr;
}
QueryPlanPtr InterpreterSelectQueryUseOptimizer::getQueryPlan(bool skip_optimize)
{
// When interpret sub query, reuse context info, e.g. PlanNodeIdAllocator, SymbolAllocator.
if (interpret_sub_query)
@ -89,68 +147,86 @@ QueryPlanPtr InterpreterSelectQueryUseOptimizer::buildQueryPlan(bool skip_optimi
return sub_query_plan;
}
AnalysisPtr analysis;
QueryPlanPtr query_plan;
UInt128 query_hash;
context->createPlanNodeIdAllocator();
context->createSymbolAllocator();
context->createOptimizerMetrics();
// not cache internal query
bool enable_plan_cache = !options.is_internal && PlanCacheManager::enableCachePlan(query_ptr, context);
// remove settings to avoid plan cache miss
RemoveSettings remove_settings_data;
RemoveSettingsVisitor(remove_settings_data).visit(query_ptr);
if (const auto * execute_query = query_ptr->as<ASTExecutePreparedStatementQuery>())
{
auto * prepared_stat_manager = context->getPreparedStatementManager();
if (!prepared_stat_manager)
throw Exception("Prepared statement cache is not initialized", ErrorCodes::LOGICAL_ERROR);
auto cache_result = prepared_stat_manager->getPlanFromCache(execute_query->name, context);
query_plan = std::move(cache_result.plan);
GraphvizPrinter::printLogicalPlan(*query_plan->getPlanNode(), context, "3997_get_prepared_plan");
PreparedParameterBindings parameter_bindings;
const auto * settings = execute_query->getValues()->as<const ASTSetQuery>();
for (const auto & change : settings->changes)
{
if (auto it = cache_result.prepared_params.find(PreparedParameter{.name = change.name});
it != cache_result.prepared_params.end())
parameter_bindings.emplace(*it, change.value);
}
PreparedStatementContext prepare_context{std::move(parameter_bindings), context};
query_plan->prepare(prepare_context);
GraphvizPrinter::printLogicalPlan(*query_plan->getPlanNode(), context, "3998_prepare_query_plan");
}
else
{
if (enable_plan_cache)
{
if (context->getSettingsRef().enable_auto_prepared_statement)
{
SubstituteLiteralToPreparedParamsMatcher::Data substitude_prepared_param_data(context);
SubstituteLiteralToPreparedParamsVisitor(substitude_prepared_param_data).visit(query_ptr);
auto_prepared_params = std::move(substitude_prepared_param_data.extracted_binding);
LOG_DEBUG(
log,
"extract auto prepared statement, SQL: {}, params: {}",
query_ptr->formatForErrorMessage(),
toString(auto_prepared_params));
}
query_hash = PlanCacheManager::hash(query_ptr, context);
query_plan = PlanCacheManager::getPlanFromCache(query_hash, context);
if (query_plan)
{
query_plan->addInterpreterContext(context);
LOG_INFO(log, "hit plan cache");
GraphvizPrinter::printLogicalPlan(*query_plan->getPlanNode(), context, "3996_get_plan_from_cache");
}
else if (context->getSettingsRef().force_plan_cache)
throw Exception(ErrorCodes::PLAN_CACHE_NOT_USED, "plan cache not used");
}
if (!query_plan)
if (!query_plan || context->getSettingsRef().iterative_optimizer_timeout == 999999)
{
Stopwatch stage_watch;
stage_watch.start();
auto cloned_query = query_ptr->clone();
cloned_query = QueryRewriter().rewrite(cloned_query, context);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Rewrite", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryRewriterTime, stage_watch.elapsedMilliseconds());
stage_watch.restart();
AnalysisPtr analysis = QueryAnalyzer::analyze(cloned_query, context);
buildQueryPlan(query_plan, analysis, skip_optimize);
GraphvizPrinter::printLogicalPlan(*query_plan->getPlanNode(), context, "3997_build_plan_from_query");
fillContextQueryAccessInfo(context, analysis);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Analyzer", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryAnalyzerTime, stage_watch.elapsedMilliseconds());
stage_watch.restart();
query_plan = QueryPlanner().plan(cloned_query, *analysis, context);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Planning", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryPlannerTime, stage_watch.elapsedMilliseconds());
if (!skip_optimize)
{
stage_watch.restart();
PlanOptimizer::optimize(*query_plan, context);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Optimizer", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryOptimizerTime, stage_watch.elapsedMilliseconds());
}
if (enable_plan_cache && query_hash && query_plan)
{
if (PlanCacheManager::addPlanToCache(query_hash, query_plan, analysis, context))
LOG_INFO(log, "plan cache added");
}
}
if (!auto_prepared_params.empty())
{
query_plan->prepare(PreparedStatementContext{auto_prepared_params, context});
GraphvizPrinter::printLogicalPlan(*query_plan->getPlanNode(), context, "3998_auto_prepare_query_plan");
}
}
if (query_plan->getPlanNodeRoot())
block = query_plan->getPlanNodeRoot()->getCurrentDataStream().header;
GraphvizPrinter::printLogicalPlan(*query_plan->getPlanNode(), context, "3999_final_plan");
query_plan->addInterpreterContext(context);
LOG_DEBUG(log, "join order {}", JoinOrderUtils::getJoinOrder(*query_plan));
return query_plan;
}
@ -170,7 +246,7 @@ std::pair<PlanSegmentTreePtr, std::set<StorageID>> InterpreterSelectQueryUseOpti
Stopwatch stage_watch, total_watch;
total_watch.start();
setUnsupportedSettings(context);
QueryPlanPtr query_plan = buildQueryPlan();
QueryPlanPtr query_plan = getQueryPlan();
query_plan->setResetStepId(false);
stage_watch.start();
@ -417,6 +493,7 @@ BlockIO InterpreterSelectQueryUseOptimizer::readFromQueryCache(ContextPtr local_
{
auto query_cache = local_context->getQueryCache();
const Settings & settings = local_context->getSettingsRef();
auto query = query_ptr->clone();
std::optional<std::set<StorageID>> used_storage_ids = getUsedStorageIds();
TxnTimestamp & source_update_time_for_query_cache = query_cache_context.source_update_time_for_query_cache;
query_cache_context.query_executed_by_optimizer = true;
@ -438,7 +515,7 @@ BlockIO InterpreterSelectQueryUseOptimizer::readFromQueryCache(ContextPtr local_
|| (source_update_time_for_query_cache.toUInt64() != 0))
{
QueryCache::Key key(
query_ptr,
query,
block,
local_context->getUserName(),
/*dummy for is_shared*/ false,
@ -463,6 +540,13 @@ BlockIO InterpreterSelectQueryUseOptimizer::readFromQueryCache(ContextPtr local_
BlockIO InterpreterSelectQueryUseOptimizer::execute()
{
if (auto * create_prepared = query_ptr->as<ASTCreatePreparedStatementQuery>())
{
// if (!create_prepared->cluster.empty())
// return executeDDLQueryOnCluster(query_ptr, context);
return executeCreatePreparedStatementQuery();
}
if (!plan_segment_tree_ptr)
{
std::pair<PlanSegmentTreePtr, std::set<StorageID>> plan_segment_tree_and_used_storage_ids = getPlanSegment();
@ -547,6 +631,85 @@ void InterpreterSelectQueryUseOptimizer::setUnsupportedSettings(ContextMutablePt
context->applySettingsChanges(setting_changes);
}
void InterpreterSelectQueryUseOptimizer::buildQueryPlan(QueryPlanPtr & query_plan, AnalysisPtr & analysis, bool skip_optimize)
{
context->createPlanNodeIdAllocator();
context->createSymbolAllocator();
context->createOptimizerMetrics();
Stopwatch stage_watch;
stage_watch.start();
query_ptr = QueryRewriter().rewrite(query_ptr, context);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Rewrite", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryRewriterTime, stage_watch.elapsedMilliseconds());
stage_watch.restart();
analysis = QueryAnalyzer::analyze(query_ptr, context);
fillContextQueryAccessInfo(context, analysis);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Analyzer", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryAnalyzerTime, stage_watch.elapsedMilliseconds());
stage_watch.restart();
query_plan = QueryPlanner().plan(query_ptr, *analysis, context);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Planning", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryPlannerTime, stage_watch.elapsedMilliseconds());
if (!skip_optimize)
{
stage_watch.restart();
PlanOptimizer::optimize(*query_plan, context);
context->logOptimizerProfile(
log, "Optimizer stage run time: ", "Optimizer", std::to_string(stage_watch.elapsedMillisecondsAsDouble()) + "ms");
ProfileEvents::increment(ProfileEvents::QueryOptimizerTime, stage_watch.elapsedMilliseconds());
}
}
BlockIO InterpreterSelectQueryUseOptimizer::executeCreatePreparedStatementQuery()
{
auto * prep_stat_manager = context->getPreparedStatementManager();
if (!prep_stat_manager)
throw Exception("Prepare cache has to be initialized", ErrorCodes::LOGICAL_ERROR);
AddDefaultDatabaseVisitor add_default_db_visitor(context, context->getCurrentDatabase());
add_default_db_visitor.visit(query_ptr);
String name;
String formatted_query;
SettingsChanges settings_changes;
const auto & prepare = query_ptr->as<const ASTCreatePreparedStatementQuery &>();
{
name = prepare.getName();
formatted_query = prepare.formatForErrorMessage();
settings_changes = InterpreterSetQuery::extractSettingsFromQuery(query_ptr, context);
}
QueryPlanPtr query_plan;
AnalysisPtr analysis;
buildQueryPlan(query_plan, analysis);
CollectPreparedParams prepared_params_collector;
CollectPreparedParamsVisitor(prepared_params_collector).visit(query_ptr);
prep_stat_manager->addPlanToCache(
name,
formatted_query,
settings_changes,
query_plan,
analysis,
std::move(prepared_params_collector.prepared_params),
context,
!prepare.if_not_exists,
prepare.or_replace,
prepare.is_permanent);
return {};
}
bool InterpreterSelectQueryUseOptimizer::isCreatePreparedStatement()
{
return query_ptr->as<ASTCreatePreparedStatementQuery>();
}
QueryPlan PlanNodeToNodeVisitor::convert(QueryPlan & query_plan)
{
QueryPlan plan;

View File

@ -14,14 +14,18 @@
*/
#pragma once
#include <Common/Stopwatch.h>
#include <Interpreters/DistributedStages/PlanSegmentSplitter.h>
#include <Interpreters/IInterpreter.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Interpreters/prepared_statement.h>
#include <QueryPlan/CTEVisitHelper.h>
#include <QueryPlan/PlanVisitor.h>
#include <Interpreters/DistributedStages/PlanSegmentSplitter.h>
#include <Interpreters/QueryLog.h>
#include <QueryPlan/QueryPlan.h>
#include <Poco/Logger.h>
#include <Common/Stopwatch.h>
namespace Poco
{
@ -38,32 +42,29 @@ class InterpreterSelectQueryUseOptimizer : public IInterpreter
{
public:
InterpreterSelectQueryUseOptimizer(const ASTPtr & query_ptr_, ContextMutablePtr context_, const SelectQueryOptions & options_)
: query_ptr(query_ptr_->clone())
, context(std::move(context_))
, options(options_)
, log(&Poco::Logger::get("InterpreterSelectQueryUseOptimizer"))
: InterpreterSelectQueryUseOptimizer(query_ptr_, nullptr, {}, context_, options_)
{
interpret_sub_query = false;
}
InterpreterSelectQueryUseOptimizer(
PlanNodePtr sub_plan_ptr_, CTEInfo cte_info_, ContextMutablePtr context_, const SelectQueryOptions & options_)
: sub_plan_ptr(sub_plan_ptr_)
, cte_info(std::move(cte_info_))
, context(std::move(context_))
, options(options_)
, log(&Poco::Logger::get("InterpreterSelectQueryUseOptimizer"))
: InterpreterSelectQueryUseOptimizer(nullptr, std::move(sub_plan_ptr_), std::move(cte_info_), context_, options_)
{
interpret_sub_query = true;
}
QueryPlanPtr buildQueryPlan(bool skip_optimize = false);
InterpreterSelectQueryUseOptimizer(
const ASTPtr & query_ptr_,
PlanNodePtr sub_plan_ptr_,
CTEInfo cte_info_,
ContextMutablePtr & context_,
const SelectQueryOptions & options_);
QueryPlanPtr getQueryPlan(bool skip_optimize = false);
void buildQueryPlan(QueryPlanPtr & query_plan, AnalysisPtr & analysis, bool skip_optimize = false);
std::pair<PlanSegmentTreePtr, std::set<StorageID>> getPlanSegment();
QueryPlanPtr getPlanFromCache(UInt128 query_hash);
bool addPlanToCache(UInt128 query_hash, QueryPlanPtr & plan, AnalysisPtr analysis);
static void setPlanSegmentInfoForExplainAnalyze(PlanSegmentTreePtr & plan_segment_tree);
BlockIO readFromQueryCache(ContextPtr local_context, QueryCacheContext & can_use_query_cache);
BlockIO execute() override;
@ -80,6 +81,8 @@ public:
static void setUnsupportedSettings(ContextMutablePtr & context);
std::optional<std::set<StorageID>> getUsedStorageIds();
BlockIO executeCreatePreparedStatementQuery();
bool isCreatePreparedStatement();
private:
ASTPtr query_ptr;
@ -94,6 +97,7 @@ private:
std::shared_ptr<std::vector<String>> segment_profiles;
Block block;
PreparedParameterBindings auto_prepared_params;
};
/**

View File

@ -1,7 +1,17 @@
#include <Parsers/ASTSetQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
namespace DB
{
@ -22,4 +32,108 @@ void InterpreterSetQuery::executeForCurrentContext()
getContext()->applySettingsChanges(ast.changes);
}
void InterpreterSetQuery::applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr setting_scontext)
{
auto changes = extractSettingsFromQuery(ast, setting_scontext);
auto new_settings = std::make_shared<ASTSetQuery>();
new_settings->changes = std::move(changes);
InterpreterSetQuery(new_settings, setting_scontext).executeForCurrentContext();
}
static SettingsChanges extractSettingsFromSetQuery(const ASTPtr & ast)
{
if (!ast)
return {};
const auto & set_query = ast->as<ASTSetQuery &>();
return set_query.changes;
}
static SettingsChanges extractSettingsFromSelectWithUnion(const ASTSelectWithUnionQuery & select_with_union)
{
auto settings = extractSettingsFromSetQuery(select_with_union.settings_ast);
const ASTs & children = select_with_union.list_of_selects->children;
if (!children.empty())
{
// We might have an arbitrarily complex UNION tree, so just give
// up if the last first-order child is not a plain SELECT.
// It is flattened later, when we process UNION ALL/DISTINCT.
const auto * last_select = children.back()->as<ASTSelectQuery>();
if (last_select && last_select->settings())
{
auto select_settings = extractSettingsFromSetQuery(last_select->settings());
settings.merge(select_settings);
}
}
return settings;
}
SettingsChanges InterpreterSetQuery::extractSettingsFromQuery(const ASTPtr & ast, ContextMutablePtr settings_context)
{
if (!ast)
return {};
if (const auto * select_query = ast->as<ASTSelectQuery>())
{
return extractSettingsFromSetQuery(select_query->settings());
}
else if (const auto * select_with_union_query = ast->as<ASTSelectWithUnionQuery>())
{
return extractSettingsFromSelectWithUnion(*select_with_union_query);
}
else if (const auto * explain_query = ast->as<ASTExplainQuery>())
{
auto settings = extractSettingsFromSetQuery(explain_query->settings_ast);
if (const auto * inner_select_union = explain_query->getExplainedQuery()->as<ASTSelectWithUnionQuery>())
{
auto inner_select_union_settings = extractSettingsFromSelectWithUnion(*inner_select_union);
settings.merge(inner_select_union_settings);
}
else if (const auto * inner_insert = explain_query->getExplainedQuery()->as<ASTInsertQuery>())
{
auto inner_insert_settings = extractSettingsFromSetQuery(inner_insert->settings_ast);
settings.merge(inner_insert_settings);
}
return settings;
}
else if (const auto * prepare_query = ast->as<ASTCreatePreparedStatementQuery>())
{
SettingsChanges settings;
if (const auto * inner_select_union = prepare_query->getQuery()->as<ASTSelectWithUnionQuery>())
{
auto inner_select_union_settings = extractSettingsFromSelectWithUnion(*inner_select_union);
settings.merge(inner_select_union_settings);
}
else if (const auto * inner_insert = prepare_query->getQuery()->as<ASTInsertQuery>())
{
auto inner_insert_settings = extractSettingsFromSetQuery(inner_insert->settings_ast);
settings.merge(inner_insert_settings);
}
return settings;
}
else if (const auto * execute_query = ast->as<ASTExecutePreparedStatementQuery>())
{
// Settings of EXECUTE PREPARED STATEMENT should include settings of corresponding CREATE PREPARED STATEMENT
auto * prepared_stat_manager = settings_context->getPreparedStatementManager();
if (!prepared_stat_manager)
throw Exception("Prepared statement cache is not initialized", ErrorCodes::LOGICAL_ERROR);
auto settings = prepared_stat_manager->getSettings(execute_query->getName());
auto execute_settings = extractSettingsFromSetQuery(execute_query->settings_ast);
settings.merge(execute_settings);
return settings;
}
else if (const auto * insert_query = ast->as<ASTInsertQuery>())
{
return extractSettingsFromSetQuery(insert_query->settings_ast);
}
else if (const auto * query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
{
return extractSettingsFromSetQuery(query_with_output->settings_ast);
}
return {};
}
}

View File

@ -2,6 +2,7 @@
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
#include <Common/SettingsChanges.h>
namespace DB
@ -25,6 +26,12 @@ public:
*/
void executeForCurrentContext();
/// To apply SETTINGS clauses from query as early as possible
static void applySettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context);
/// To extract SETTINGS clauses from query
static SettingsChanges extractSettingsFromQuery(const ASTPtr & ast, ContextMutablePtr context);
private:
ASTPtr query_ptr;
};

View File

@ -0,0 +1,76 @@
#include <string>
#include <Columns/ColumnString.h>
#include <DataStreams/OneBlockInputStream.h>
#include <DataTypes/DataTypeString.h>
#include <Interpreters/InterpreterShowPreparedStatementQuery.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/formatAST.h>
#include <QueryPlan/PlanPrinter.h>
namespace DB
{
BlockIO InterpreterShowPreparedStatementQuery::execute()
{
const auto * show_prepared = query_ptr->as<const ASTShowPreparedStatementQuery>();
// get prepare from cache
if (!context->getPreparedStatementManager())
throw Exception("Prepare cache has to be initialized", ErrorCodes::LOGICAL_ERROR);
if ((show_prepared->show_create || show_prepared->show_explain) && show_prepared->name.empty())
throw Exception("No preapred statement name specified", ErrorCodes::LOGICAL_ERROR);
std::ostringstream out;
String result_column_name;
auto * prepared_manager = context->getPreparedStatementManager();
if (show_prepared->show_create)
{
auto prepared_object = prepared_manager->getObject(show_prepared->name);
out << prepared_object.query;
result_column_name = "Create Statement";
}
else if (show_prepared->show_explain)
{
auto prepared_object = prepared_manager->getObject(show_prepared->name);
CTEInfo cte_info;
for (auto & cte_it : prepared_object.cte_map)
cte_info.add(cte_it.first, cte_it.second);
QueryPlan plan(prepared_object.plan_root, cte_info, context->getPlanNodeIdAllocator());
out << PlanPrinter::textLogicalPlan(plan, context, false, true, {});
result_column_name = "Explain";
}
// else if (!show_prepared->name.empty())
// {
// auto prepared_object = prepared_manager->getObject(show_prepared->name);
// CTEInfo cte_info;
// for (auto & cte_it : prepared_object.cte_map)
// cte_info.add(cte_it.first, cte_it.second);
// QueryPlan plan(prepared_object.plan_root, cte_info, context->getPlanNodeIdAllocator());
// out << "Prepare Name: " << show_prepared->name << "\n"
// << "Create Statement:\n"
// << prepared_object.query << ";\n"
// << "Explain:\n"
// << PlanPrinter::textLogicalPlan(plan, context, false, true, {});
// result_column_name = "Prepared Statement Detail";
// }
else
{
auto name_list = prepared_manager->getNames();
for (auto & name : name_list)
out << name << "\n";
result_column_name = "Prepared Statement List";
}
BlockIO res;
MutableColumnPtr binding_column = ColumnString::create();
std::istringstream ss(out.str());
std::string line;
while (std::getline(ss, line))
binding_column->insert(std::move(line));
res.in
= std::make_shared<OneBlockInputStream>(Block{{std::move(binding_column), std::make_shared<DataTypeString>(), result_column_name}});
return res;
}
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Parsers/IAST_fwd.h>
#include <utility>
namespace DB
{
class InterpreterShowPreparedStatementQuery : public IInterpreter
{
public:
explicit InterpreterShowPreparedStatementQuery(const ASTPtr & query_ptr_, ContextMutablePtr & context_)
: query_ptr(query_ptr_), context(context_)
{
}
BlockIO execute() override;
private:
ASTPtr query_ptr;
ContextMutablePtr context;
};
}

View File

@ -0,0 +1,199 @@
#include <Interpreters/PreparedStatement/PreparedStatementLoaderFromDisk.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/atomicRename.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <common/logger_useful.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Protos/plan_node.pb.h>
#include <Poco/DirectoryIterator.h>
#include <Poco/Logger.h>
#include <filesystem>
#include <memory>
namespace fs = std::filesystem;
namespace DB
{
namespace ErrorCodes
{
extern const int DIRECTORY_DOESNT_EXIST;
extern const int PREPARED_STATEMENT_ALREADY_EXISTS;
extern const int PREPARED_STATEMENT_NOT_EXISTS;
}
namespace
{
/// Converts a path to an absolute path and append it with a separator.
String makeDirectoryPathCanonical(const String & directory_path)
{
auto canonical_directory_path = std::filesystem::weakly_canonical(directory_path);
if (canonical_directory_path.has_filename())
canonical_directory_path += std::filesystem::path::preferred_separator;
return canonical_directory_path;
}
}
PreparedStatementLoaderFromDisk::PreparedStatementLoaderFromDisk(const String & dir_path_)
: dir_path{makeDirectoryPathCanonical(dir_path_)}, log{&Poco::Logger::get("PreparedStatementLoaderFromDisk")}
{
createDirectory();
}
std::optional<Protos::PreparedStatement> PreparedStatementLoaderFromDisk::tryGetPreparedObject(
const String & statement_name, const String & path, ContextPtr /*context*/, bool check_file_exists)
{
std::shared_lock lock(mutex);
LOG_DEBUG(log, "Loading prepared statement {} from file {}", backQuote(statement_name), path);
try
{
if (check_file_exists && !fs::exists(path))
return std::nullopt;
std::ifstream fin(path, std::ios::binary);
Protos::PreparedStatement pb;
pb.ParseFromIstream(&fin);
return pb;
}
catch (...)
{
tryLogCurrentException(
log, fmt::format("while loading prepared statement SQL object {} from path {}", backQuote(statement_name), path));
return std::nullopt; /// Failed to load this sql object, will ignore it
}
}
NamesAndPreparedStatements PreparedStatementLoaderFromDisk::getAllObjects(ContextPtr context)
{
LOG_DEBUG(log, "Loading Prepared Statements from {}", dir_path);
NamesAndPreparedStatements statements;
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator it(dir_path); it != dir_end; ++it)
{
if (it->isDirectory())
continue;
const String & file_name = it.name();
if (!endsWith(file_name, ".bin"))
continue;
size_t prefix_length;
if (startsWith(file_name, "prepared_"))
{
prefix_length = strlen("prepared_");
}
else
{
continue;
}
size_t suffix_length = strlen(".bin");
String statement_name = unescapeForFileName(file_name.substr(prefix_length, file_name.length() - prefix_length - suffix_length));
if (statement_name.empty())
continue;
auto statement = tryGetPreparedObject(statement_name, dir_path + it.name(), context, /* check_file_exists= */ false);
if (!statement)
continue;
statements.emplace_back(std::move(statement_name), std::move(*statement));
}
LOG_DEBUG(log, "Prepared Statements loaded");
return statements;
}
void PreparedStatementLoaderFromDisk::createDirectory()
{
std::error_code create_dir_error_code;
fs::create_directories(dir_path, create_dir_error_code);
if (!fs::exists(dir_path) || !fs::is_directory(dir_path) || create_dir_error_code)
throw Exception(
"Couldn't create directory " + dir_path + " reason: '" + create_dir_error_code.message() + "'",
ErrorCodes::DIRECTORY_DOESNT_EXIST);
}
bool PreparedStatementLoaderFromDisk::storeObject(
const String & statement_name, const Protos::PreparedStatement & prepared_statement, bool throw_if_exists, bool replace_if_exists)
{
std::unique_lock lock(mutex);
String file_path = getFilePath(statement_name);
LOG_DEBUG(log, "Storing Prepared Statement {} to file {}", backQuote(statement_name), file_path);
if (fs::exists(file_path))
{
if (throw_if_exists)
throw Exception(ErrorCodes::PREPARED_STATEMENT_ALREADY_EXISTS, "Prepared Statement '{}' already exists", statement_name);
else if (!replace_if_exists)
return false;
}
String temp_file_path = file_path + ".tmp";
try
{
std::ofstream fout(temp_file_path, std::ios::binary);
prepared_statement.SerializeToOstream(&fout);
if (replace_if_exists)
fs::rename(temp_file_path, file_path);
else
renameNoReplace(temp_file_path, file_path);
}
catch (...)
{
fs::remove(temp_file_path);
throw;
}
LOG_DEBUG(log, "Prepared Statement {} stored", backQuote(statement_name));
return true;
}
bool PreparedStatementLoaderFromDisk::removeObject(const String & statement_name, bool throw_if_not_exists)
{
std::unique_lock lock(mutex);
String file_path = getFilePath(statement_name);
LOG_DEBUG(log, "Removing Prepared Statement object {} stored in file {}", backQuote(statement_name), file_path);
bool existed = fs::remove(file_path);
if (!existed)
{
if (throw_if_not_exists)
throw Exception(ErrorCodes::PREPARED_STATEMENT_NOT_EXISTS, "Prepared Statement '{}' doesn't exist", statement_name);
else
return false;
}
LOG_DEBUG(log, "Prepared Statement {} removed", backQuote(statement_name));
return true;
}
String PreparedStatementLoaderFromDisk::getFilePath(const String & statement_name) const
{
String file_path = dir_path + "prepared_" + statement_name + ".bin";
return file_path;
}
}

View File

@ -0,0 +1,42 @@
#pragma once
#include <Core/Settings.h>
#include <Interpreters/Context_fwd.h>
#include <Parsers/IAST_fwd.h>
#include <shared_mutex>
namespace DB
{
namespace Protos
{
class PreparedStatement;
}
using NamesAndPreparedStatements = std::vector<std::pair<String, Protos::PreparedStatement>>;
class PreparedStatementLoaderFromDisk
{
public:
explicit PreparedStatementLoaderFromDisk(const String & dir_path_);
bool storeObject(
const String & statement_name, const Protos::PreparedStatement & prepared_statement, bool throw_if_exists, bool replace_if_exists);
bool removeObject(const String & statement_name, bool throw_if_not_exists);
NamesAndPreparedStatements getAllObjects(ContextPtr context);
private:
void createDirectory();
std::optional<Protos::PreparedStatement>
tryGetPreparedObject(const String & statement_name, const String & file_path, ContextPtr context, bool check_file_exists);
String getFilePath(const String & statement_name) const;
String dir_path;
mutable std::shared_mutex mutex;
Poco::Logger * log;
};
}

View File

@ -0,0 +1,246 @@
#include <DataTypes/Serializations/ISerialization.h>
#include <Interpreters/InterpreterSelectQueryUseOptimizer.h>
#include <Interpreters/PreparedStatement/PreparedStatementManager.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ParserPreparedStatement.h>
#include <Parsers/parseQuery.h>
#include <Protos/plan_node.pb.h>
#include <memory>
namespace DB
{
namespace ErrorCodes
{
extern const int PREPARED_STATEMENT_ALREADY_EXISTS;
extern const int PREPARED_STATEMENT_NOT_EXISTS;
}
void PreparedObject::toProto(Protos::PreparedStatement & proto) const
{
proto.set_query(query);
}
void PreparedStatementManager::initialize(ContextMutablePtr context)
{
if (!context->getPreparedStatementManager())
{
auto manager_instance = std::make_unique<PreparedStatementManager>();
const auto & config = context->getConfigRef();
String default_path = fs::path{context->getPath()} / "prepared_statement/";
String path = config.getString("prepared_statement_path", default_path);
manager_instance->prepared_statement_loader = std::make_unique<PreparedStatementLoaderFromDisk>(path);
context->setPreparedStatementManager(std::move(manager_instance));
loadStatementsFromDisk(context);
}
}
void PreparedStatementManager::set(
const String & name, PreparedObject prepared_object, bool throw_if_exists, bool or_replace, bool is_persistent)
{
std::unique_lock lock(mutex);
if (!hasUnsafe(name) || or_replace)
{
if (is_persistent)
{
Protos::PreparedStatement proto;
prepared_object.toProto(proto);
throw_if_exists = throw_if_exists && !or_replace;
prepared_statement_loader->storeObject(name, proto, throw_if_exists, or_replace);
}
cache[name] = std::move(prepared_object);
}
else if (throw_if_exists)
throw Exception(ErrorCodes::PREPARED_STATEMENT_ALREADY_EXISTS, "Prepared statement already exists");
}
PreparedObject PreparedStatementManager::getObject(const String & name) const
{
std::shared_lock lock(mutex);
return getUnsafe(name);
}
SettingsChanges PreparedStatementManager::getSettings(const String & name) const
{
std::shared_lock lock(mutex);
return getUnsafe(name).settings_changes;
}
void PreparedStatementManager::remove(const String & name, bool throw_if_not_exists)
{
std::unique_lock lock(mutex);
prepared_statement_loader->removeObject(name, false);
if (hasUnsafe(name))
cache.erase(name);
else if (throw_if_not_exists)
throw Exception(ErrorCodes::PREPARED_STATEMENT_NOT_EXISTS, "Prepared statement not exists");
}
PreparedStatementManager::CacheResultType PreparedStatementManager::getPlanFromCache(const String & name, ContextMutablePtr & context) const
{
std::shared_lock lock(mutex);
const auto & prepared_object = getUnsafe(name);
PlanNodeId max_id;
auto root = getNewPlanNode(prepared_object.plan_root, context, false, max_id);
CTEInfo cte_info;
for (const auto & cte : prepared_object.cte_map)
cte_info.add(cte.first, getNewPlanNode(cte.second, context, false, max_id));
if (prepared_object.query_detail && context->hasQueryContext())
{
for (auto & [database, table_info] : prepared_object.query_detail->query_access_info)
{
for (auto & [table, columns] : table_info)
context->addQueryAccessInfo(database, table, columns);
}
}
auto node_id_allocator = std::make_shared<PlanNodeIdAllocator>(max_id + 1);
auto query_plan = std::make_unique<QueryPlan>(root, cte_info, node_id_allocator);
return {.plan = std::move(query_plan), .prepared_params = prepared_object.prepared_params};
}
void PreparedStatementManager::addPlanToCache(
const String & name,
const String & query,
SettingsChanges settings_changes,
QueryPlanPtr & plan,
AnalysisPtr analysis,
PreparedParameterSet prepared_params,
ContextMutablePtr & context,
bool throw_if_exists,
bool or_replace,
bool is_persistent)
{
PlanNodeId max_id;
PreparedObject prepared_object{};
prepared_object.query = query;
prepared_object.settings_changes = std::move(settings_changes);
prepared_object.prepared_params = std::move(prepared_params);
prepared_object.plan_root = getNewPlanNode(plan->getPlanNode(), context, true, max_id);
for (const auto & cte : plan->getCTEInfo().getCTEs())
prepared_object.cte_map.emplace(cte.first, getNewPlanNode(cte.second, context, true, max_id));
prepared_object.query_detail = std::make_shared<PreparedObject::QueryAccessInfo>();
const auto & used_columns_map = analysis->getUsedColumns();
for (const auto & [table_ast, storage_analysis] : analysis->getStorages())
{
if (!storage_analysis.storage)
continue;
auto storage_id = storage_analysis.storage->getStorageID();
if (auto it = used_columns_map.find(storage_analysis.storage->getStorageID()); it != used_columns_map.end())
{
for (const auto & column : it->second)
prepared_object.query_detail
->query_access_info[backQuoteIfNeed(storage_id.getDatabaseName())][storage_id.getFullTableName()]
.emplace_back(column);
}
}
set(name, std::move(prepared_object), throw_if_exists, or_replace, is_persistent);
}
PlanNodePtr PreparedStatementManager::getNewPlanNode(PlanNodePtr node, ContextMutablePtr & context, bool cache_plan, PlanNodeId & max_id)
{
if (max_id < node->getId())
max_id = node->getId();
if (node->getType() == IQueryPlanStep::Type::TableScan)
{
auto step = node->getStep()->copy(context);
auto * table_step = dynamic_cast<TableScanStep *>(step.get());
if (cache_plan)
table_step->cleanStorage();
else
table_step->setStorage(context);
return PlanNodeBase::createPlanNode(node->getId(), step, {});
}
PlanNodes children;
for (auto & child : node->getChildren())
{
auto result_node = getNewPlanNode(child, context, cache_plan, max_id);
if (result_node)
children.emplace_back(result_node);
}
return PlanNodeBase::createPlanNode(node->getId(), node->getStep()->copy(context), children);
}
const PreparedObject & PreparedStatementManager::getUnsafe(const String & name) const
{
auto it = cache.find(name);
if (it == cache.end())
throw Exception(ErrorCodes::PREPARED_STATEMENT_NOT_EXISTS, "Prepared statement not exists");
return it->second;
}
Strings PreparedStatementManager::getNames() const
{
std::shared_lock lock(mutex);
Strings res;
res.reserve(cache.size());
for (const auto & elem : cache)
res.push_back(elem.first);
return res;
}
bool PreparedStatementManager::has(const String & name) const
{
std::shared_lock lock(mutex);
bool contains = hasUnsafe(name);
return contains;
}
void PreparedStatementManager::clearCache()
{
std::unique_lock lock(mutex);
cache.clear();
}
NamesAndPreparedStatements PreparedStatementManager::getAllStatementsFromDisk(ContextMutablePtr & context)
{
return prepared_statement_loader->getAllObjects(context);
}
void PreparedStatementManager::loadStatementsFromDisk(ContextMutablePtr & context)
{
if (!context->getPreparedStatementManager())
throw Exception("PreparedStatement cache has to be initialized", ErrorCodes::LOGICAL_ERROR);
auto * manager = context->getPreparedStatementManager();
manager->clearCache();
auto statements = manager->getAllStatementsFromDisk(context);
for (auto & statement : statements)
{
try
{
ParserCreatePreparedStatementQuery parser(ParserSettings::valueOf(context->getSettingsRef()));
auto ast = parseQuery(parser, statement.second.query(), "", 0, context->getSettings().max_parser_depth);
auto * create_prep_stat = ast->as<ASTCreatePreparedStatementQuery>();
if (!create_prep_stat)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid prepread statement query: {}", statement.second.query());
create_prep_stat->is_permanent = false;
InterpreterSelectQueryUseOptimizer interpreter{ast, context, {}};
interpreter.executeCreatePreparedStatementQuery();
}
catch (...)
{
tryLogWarningCurrentException(
&Poco::Logger::get("PreparedStatementManager"),
fmt::format("while build prepared statement {} plan", backQuote(statement.first)));
continue;
}
}
}
}

View File

@ -0,0 +1,104 @@
#pragma once
#include <Analyzers/Analysis.h>
#include <Interpreters/Context.h>
#include <Interpreters/PreparedStatement/PreparedStatementLoaderFromDisk.h>
#include <Interpreters/prepared_statement.h>
#include <QueryPlan/PlanNode.h>
#include <QueryPlan/QueryPlan.h>
#include <Common/SettingsChanges.h>
#include <memory>
#include <shared_mutex>
#include <unordered_map>
namespace DB
{
namespace Protos
{
class PreparedStatement;
}
using NamesAndPreparedStatements = std::vector<std::pair<String, Protos::PreparedStatement>>;
struct PreparedObject
{
struct QueryAccessInfo
{
// database_name->table_name->column_names
std::unordered_map<String, std::unordered_map<String, std::vector<String>>> query_access_info;
};
String query;
SettingsChanges settings_changes;
PreparedParameterSet prepared_params;
std::shared_ptr<QueryAccessInfo> query_detail;
PlanNodePtr plan_root;
std::unordered_map<CTEId, PlanNodePtr> cte_map;
void toProto(Protos::PreparedStatement & proto) const;
};
class PreparedStatementManager
{
public:
using CacheType = std::unordered_map<String, PreparedObject>;
static void initialize(ContextMutablePtr context);
void
set(const String & name,
PreparedObject prepared_object,
bool throw_if_exists = true,
bool or_replace = false,
bool is_persistent = true);
PreparedObject getObject(const String & name) const;
SettingsChanges getSettings(const String & name) const;
void remove(const String & name, bool throw_if_not_exists);
void clearCache();
Strings getNames() const;
bool has(const String & name) const;
NamesAndPreparedStatements getAllStatementsFromDisk(ContextMutablePtr & context);
struct CacheResultType
{
QueryPlanPtr plan;
PreparedParameterSet prepared_params;
};
// TODO @wangtao: extract common logic with InterpreterSelectQueryUseOptimizer::getPlanFromCache
CacheResultType getPlanFromCache(const String & name, ContextMutablePtr & context) const;
// TODO @wangtao: extract common logic with InterpreterSelectQueryUseOptimizer::addPlanToCache
void addPlanToCache(
const String & name,
const String & query,
SettingsChanges settings_changes,
QueryPlanPtr & plan,
AnalysisPtr analysis,
PreparedParameterSet prepared_params,
ContextMutablePtr & context,
bool throw_if_exists,
bool or_replace,
bool is_persistent);
static void loadStatementsFromDisk(ContextMutablePtr & context);
private:
CacheType cache;
mutable std::shared_mutex mutex;
std::unique_ptr<PreparedStatementLoaderFromDisk> prepared_statement_loader;
bool hasUnsafe(const String & name) const
{
return cache.contains(name);
}
// TODO @wangtao: extract common logic with PlanCache::getNewPlanNode
static PlanNodePtr getNewPlanNode(PlanNodePtr node, ContextMutablePtr & context, bool cache_plan, PlanNodeId & max_id);
const PreparedObject & getUnsafe(const String & name) const;
};
}

View File

@ -51,9 +51,14 @@
#include <Processors/Sources/RemoteSource.h>
#include <Processors/Transforms/getSourceFromFromASTInsertQuery.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTRenameQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
@ -62,6 +67,7 @@
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/Lexer.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryNormalization.h>
#include <Parsers/queryToString.h>
@ -874,21 +880,21 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
ParserQuery parser(end, ParserSettings::valueOf(context->getSettings()));
parser.setContext(context.get());
/// TODO Parser should fail early when max_query_size limit is reached.
ast = parseQuery(parser, begin, end, "", max_query_size, context->getSettings().max_parser_depth);
if (settings.use_sql_binding && !internal)
{
try
/// TODO Parser should fail early when max_query_size limit is reached.
ast = parseQuery(parser, begin, end, "", max_query_size, context->getSettings().max_parser_depth);
if (settings.use_sql_binding && !internal)
{
ASTPtr binding_ast = SQLBindingUtils::getASTFromBindings(begin, end, ast, context);
if (binding_ast)
ast = binding_ast;
try
{
ASTPtr binding_ast = SQLBindingUtils::getASTFromBindings(begin, end, ast, context);
if (binding_ast)
ast = binding_ast;
}
catch (...)
{
tryLogWarningCurrentException(&Poco::Logger::get("SQL Binding"), "SQL binding match error.");
}
}
catch (...)
{
tryLogWarningCurrentException(&Poco::Logger::get("SQL Binding"), "SQL binding match error.");
}
}
}
else
{
@ -928,7 +934,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Interpret SETTINGS clauses as early as possible (before invoking the corresponding interpreter),
/// to allow settings to take effect.
if (input_ast == nullptr)
interpretSettings(ast, context);
InterpreterSetQuery::applySettingsFromQuery(ast, context);
if (context->getServerType() == ServerType::cnch_server && context->hasQueryContext())
{
@ -938,22 +944,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
IdentifierNameNormalizer().visit<false>(ast.get());
}
if (auto * explain_select_query = ast->as<ASTExplainQuery>())
{
const auto * select_with_union_query = explain_select_query->getExplainedQuery()->as<ASTSelectWithUnionQuery>();
if (select_with_union_query && !select_with_union_query->list_of_selects->children.empty())
{
const auto * last_select = select_with_union_query->list_of_selects->children.back()->as<ASTSelectQuery>();
if (last_select && last_select->settings())
InterpreterSetQuery(last_select->settings(), context).executeForCurrentContext();
}
else
{
auto * insert_query = explain_select_query->getExplainedQuery()->as<ASTInsertQuery>();
if (insert_query && insert_query->settings_ast)
InterpreterSetQuery(insert_query->settings_ast, context).executeForCurrentContext();
}
}
/// Interpret SETTINGS clauses as early as possible (before invoking the corresponding interpreter),
/// to allow settings to take effect.
if (const auto * query_with_table_output = dynamic_cast<const ASTQueryWithTableAndOutput *>(ast.get()))
{
@ -961,10 +953,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
query_table = query_with_table_output->table;
}
auto * insert_query = ast->as<ASTInsertQuery>();
context->setQueryExpirationTimeStamp();
auto * insert_query = ast->as<ASTInsertQuery>();
if (insert_query && insert_query->data)
{
query_end = insert_query->data;
@ -1164,7 +1154,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
else
{
InterpreterSelectQueryUseOptimizer * optimizer_interpret = typeid_cast<InterpreterSelectQueryUseOptimizer *>(&*interpreter);
if (optimizer_interpret)
if (optimizer_interpret && !optimizer_interpret->isCreatePreparedStatement())
{
res = optimizer_interpret->readFromQueryCache(context, query_cache_context);
if (query_cache_context.query_cache_usage != QueryCache::Usage::Read)
@ -1176,6 +1166,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
catch (...)
{
if (ast->as<ASTExecutePreparedStatementQuery>() || ast->as<ASTCreatePreparedStatementQuery>())
throw;
if (typeid_cast<const InterpreterSelectQueryUseOptimizer *>(&*interpreter))
{
static std::unordered_set<int> no_fallback_error_codes = {159, 202, 209, 252, 394, 2010, 2012, 2013, 1159, 241};

View File

@ -0,0 +1,76 @@
#include <Interpreters/prepared_statement.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTPreparedParameter.h>
#include <Common/FieldVisitorConvertToNumber.h>
#include <Common/FieldVisitorToString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_PREPARED_PARAMETER;
}
namespace
{
struct ReplacePreparedParameter
{
using TypeToVisit = ASTPreparedParameter;
const PreparedStatementContext & prepared_context;
void visit(ASTPreparedParameter & prepared_param, ASTPtr & ast) const
{
const auto & param_name = prepared_param.name;
ast = std::make_shared<ASTLiteral>(prepared_context.getParamValue(param_name));
}
};
using ReplacePreparedParameterVisitor = InDepthNodeVisitor<OneTypeMatcher<ReplacePreparedParameter>, true>;
}
String toString(const PreparedParameterBindings & binding)
{
String str;
for (const auto & [param, value] : binding)
str += param.toString() + " = " + applyVisitor(FieldVisitorToString(), value) + ", ";
return str;
}
Field PreparedStatementContext::getParamValue(const String & param_name) const
{
auto it = param_bindings.find(PreparedParameter{.name = param_name});
if (it == param_bindings.end())
throw Exception(ErrorCodes::BAD_PREPARED_PARAMETER, "Unresolved prepare parameter {}", param_name);
return it->second;
}
void PreparedStatementContext::prepare(ASTPtr & ast) const
{
if (!ast)
return;
ReplacePreparedParameter data{*this};
ReplacePreparedParameterVisitor(data).visit(ast);
}
void PreparedStatementContext::prepare(ConstASTPtr & ast) const
{
ASTPtr ptr = std::const_pointer_cast<IAST>(ast);
prepare(ptr);
ast = ptr;
}
void PreparedStatementContext::prepare(SizeOrVariable & size_or_variable) const
{
if (!std::holds_alternative<String>(size_or_variable))
return;
const auto & param_name = std::get<String>(size_or_variable);
// TODO: prevent negative number converted to UInt64
UInt64 val = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), getParamValue(param_name));
size_or_variable = size_t{val};
}
}

View File

@ -0,0 +1,66 @@
#pragma once
#include <Core/Field.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/Context_fwd.h>
#include <Parsers/IAST_fwd.h>
#include <common/variant_helper.h>
#include <fmt/format.h>
#include <unordered_map>
#include <unordered_set>
#include <variant>
namespace DB
{
// types for data which can be a prepared param
using SizeOrVariable = std::variant<size_t, String>;
using UInt64OrVariable = std::variant<UInt64, String>;
struct PreparedParameter
{
String name;
DataTypePtr type;
struct Hash
{
std::size_t operator()(const PreparedParameter & e) const
{
return std::hash<String>()(e.name);
}
};
bool operator==(const PreparedParameter & o) const
{
return name == o.name;
}
String toString() const
{
return fmt::format("[{}:{}]", name, type->getName());
}
};
using PreparedParameterSet = std::unordered_set<PreparedParameter, PreparedParameter::Hash>;
using PreparedParameterBindings = std::unordered_map<PreparedParameter, Field, PreparedParameter::Hash>;
String toString(const PreparedParameterBindings & binding);
class PreparedStatementContext : public WithContext
{
public:
explicit PreparedStatementContext(PreparedParameterBindings param_bindings_, const ContextPtr & context_)
: WithContext(context_), param_bindings(std::move(param_bindings_))
{
}
Field getParamValue(const String & param_name) const;
void prepare(ASTPtr & ast) const;
void prepare(ConstASTPtr & ast) const;
void prepare(SizeOrVariable & size_or_variable) const;
private:
PreparedParameterBindings param_bindings;
};
}

View File

@ -19,8 +19,7 @@ namespace DB
{
PlanNodeStatisticsPtr LimitEstimator::estimate(PlanNodeStatisticsPtr & child_stats, const LimitStep & step)
{
size_t limit = step.getLimit();
return getLimitStatistics(child_stats, limit);
return step.hasPreparedParam() ? child_stats : getLimitStatistics(child_stats, step.getLimitValue());
}
PlanNodeStatisticsPtr LimitEstimator::estimate(PlanNodeStatisticsPtr & child_stats, const LimitByStep & step)

View File

@ -5,9 +5,9 @@ namespace DB
{
PlanNodeStatisticsPtr SortingEstimator::estimate(PlanNodeStatisticsPtr & child_stats, const SortingStep & step)
{
if (step.getLimit() > 0)
if (!step.hasPreparedParam() && step.getLimitValue() > 0)
{
size_t limit = step.getLimit();
size_t limit = step.getLimitValue();
return LimitEstimator::getLimitStatistics(child_stats, limit);
}
return child_stats;

View File

@ -42,7 +42,8 @@ bool IsDistinctPlanVisitor::visitValuesNode(ValuesNode & node, Void &)
bool IsDistinctPlanVisitor::visitLimitNode(LimitNode & node, Void &)
{
return dynamic_cast<const LimitStep *>(node.getStep().get())->getLimit() <= 1;
const auto * step = dynamic_cast<const LimitStep *>(node.getStep().get());
return !step->hasPreparedParam() && step->getLimitValue() <= 1;
}
bool IsDistinctPlanVisitor::visitIntersectNode(IntersectNode & node, Void & context)

View File

@ -487,6 +487,8 @@ InterpretIMResult ExpressionInterpreter::visit(const ConstASTPtr & node) const
return visitASTLiteral(*ast_literal, node);
if (const auto * ast_identifier = node->as<ASTIdentifier>())
return visitASTIdentifier(*ast_identifier, node);
if (const auto * ast_prepared_param = node->as<ASTPreparedParameter>())
return visitASTPreparedParameter(*ast_prepared_param, node);
if (const auto * ast_func = node->as<ASTFunction>())
{
const auto & func_name = ast_func->name;
@ -514,6 +516,11 @@ InterpretIMResult ExpressionInterpreter::visitASTIdentifier(const ASTIdentifier
return originalNode(node);
}
InterpretIMResult ExpressionInterpreter::visitASTPreparedParameter(const ASTPreparedParameter &, const ConstASTPtr & node) const
{
return originalNode(node);
}
InterpretIMResult ExpressionInterpreter::visitOrdinaryFunction(const ASTFunction & function, const ConstASTPtr & node) const
{
InterpretIMResults argument_results;

View File

@ -167,6 +167,7 @@ private:
InterpretIMResult visit(const ConstASTPtr & node) const;
InterpretIMResult visitASTLiteral(const ASTLiteral & literal, const ConstASTPtr & node) const;
InterpretIMResult visitASTIdentifier(const ASTIdentifier & identifier, const ConstASTPtr & node) const;
InterpretIMResult visitASTPreparedParameter(const ASTPreparedParameter & prepared_param, const ConstASTPtr & node) const;
InterpretIMResult visitOrdinaryFunction(const ASTFunction & function, const ConstASTPtr & node) const;
InterpretIMResult visitInFunction(const ASTFunction & function, const ConstASTPtr & node) const;

View File

@ -104,7 +104,7 @@ protected:
bool visitSortingNode(SortingNode & node, ContextMutablePtr & context) override
{
return dynamic_cast<const SortingStep *>(node.getStep().get())->getLimit() == 0 && visitChildren(node, context);
return dynamic_cast<const SortingStep *>(node.getStep().get())->getLimitValue() == 0 && visitChildren(node, context);
}
private:

View File

@ -43,7 +43,8 @@ public:
{
auto source_range = VisitorUtil::accept(node.getChildren()[0], *this, context);
const auto * step = dynamic_cast<const LimitStep *>(node.getStep().get());
return applyLimit(applyOffset(source_range, step->getOffset()), step->getLimit());
return step->hasPreparedParam() ? source_range
: applyLimit(applyOffset(source_range, step->getOffsetValue()), step->getLimitValue());
}
Range visitProjectionNode(ProjectionNode & node, Void & context) override

View File

@ -41,21 +41,34 @@ Constants ConstantsDeriverVisitor::visitFilterStep(const FilterStep & step, Cons
Predicate::ExtractionResult<String> result
= translator.getExtractionResult(step.getFilter()->clone(), step.getOutputStream().header.getNamesAndTypes());
auto values = result.tuple_domain.extractFixedValues();
if (values.has_value())
std::map<String, FieldWithType> filter_values;
const Constants & origin_constants = context.getInput()[0];
for (const auto & value : origin_constants.getValues())
{
filter_values[value.first] = value.second;
}
if (values)
{
std::map<String, FieldWithType> filter_values;
const Constants & origin_constants = context.getInput()[0];
for (const auto & value : origin_constants.getValues())
{
filter_values[value.first] = value.second;
}
for (auto & value : values.value())
{
filter_values[value.first] = value.second;
}
return Constants{filter_values};
}
return context.getInput()[0];
// tmpfix for prepared params
for (const auto & conjunct : PredicateUtils::extractConjuncts(step.getFilter()->clone()))
{
const auto * func = conjunct->as<ASTFunction>();
if (!func || func->name != "equals")
continue;
const auto * column = func->arguments->children[0]->as<ASTIdentifier>();
if (!column)
continue;
const auto * prepared_param = func->arguments->children[1]->as<ASTPreparedParameter>();
if (!prepared_param)
continue;
filter_values[column->name()] = FieldWithType{DataTypeFactory::instance().get(prepared_param->type), String{prepared_param->name}};
}
return Constants{filter_values};
}
Constants ConstantsDeriverVisitor::visitJoinStep(const JoinStep & step, ConstantsDeriverContext & context)

View File

@ -22,6 +22,7 @@
#include <Interpreters/getTableExpressions.h>
#include <Interpreters/misc.h>
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTSelectIntersectExceptQuery.h>
#include <Parsers/ASTWithElement.h>
#include <Storages/StorageView.h>
@ -135,6 +136,11 @@ bool QueryUseOptimizerChecker::check(ASTPtr node, ContextMutablePtr context, boo
|| explain->getKind() == ASTExplainQuery::MetaData;
return explain_plan && check(explain->getExplainedQuery(), context, throw_exception);
}
if (auto * prepare = node->as<ASTCreatePreparedStatementQuery>())
{
return check(prepare->getQuery(), context, throw_exception);
}
bool support = false;

View File

@ -1133,5 +1133,113 @@ PlanNodePtr ColumnPruningVisitor::convertDistinctToGroupBy(PlanNodePtr node, Con
return node;
}
/*
PlanNodePtr ColumnPruningVisitor::convertFilterWindowToSortingLimit(PlanNodePtr node, NameSet & require, ContextMutablePtr & context)
{
const auto & filter_step = dynamic_cast<FilterStep &>(*node->getStep());
auto * window_node = dynamic_cast<WindowNode *>(node->getChildren()[0].get());
if (!window_node)
return node;
const auto & window_step = dynamic_cast<WindowStep &>(*window_node->getStep());
const auto & window_desc = window_step.getWindow();
if (window_desc.order_by.empty() || !window_desc.partition_by.empty() || window_desc.window_functions.size() != 1)
return node;
String column_name = window_desc.window_functions[0].column_name;
if (require.contains(column_name))
return node;
if (window_desc.frame.begin_type != WindowFrame::BoundaryType::Unbounded
|| (window_desc.frame.end_type != WindowFrame::BoundaryType::Current
&& window_desc.frame.end_type != WindowFrame::BoundaryType::Unbounded))
return node;
auto window_func_name = window_desc.window_functions[0].aggregate_function->getName();
if (window_func_name != "row_number")
return node;
const auto & filter = filter_step.getFilter();
auto symbols = SymbolsExtractor::extractVector(filter);
if (std::count(symbols.begin(), symbols.end(), column_name) != 1)
return node;
PlanNodePtr window_child = window_node->getChildren()[0];
auto interpreter = ExpressionInterpreter::basicInterpreter(filter_step.getInputStreams()[0].header.getNamesToTypes(), context);
auto conjuncts = PredicateUtils::extractConjuncts(filter);
std::vector<ConstASTPtr> new_conjuncts;
SizeOrVariable limit;
String func_name;
auto get_limit_field = [&](const ConstASTPtr & conjunct) -> std::optional<SizeOrVariable> {
func_name = "";
const auto * func = conjunct->as<ASTFunction>();
if (!func || func->arguments->children.size() != 2 || !(func->name == "less" || func->name == "lessOrEquals"))
return std::nullopt;
func_name = func->name;
const auto * column = func->arguments->children[0]->as<ASTIdentifier>();
if (!column || column_name != column->name())
return std::nullopt;
if (const auto * prepared_param = func->arguments->children[1]->as<ASTPreparedParameter>())
return prepared_param->name;
auto rhs = interpreter.evaluateConstantExpression(func->arguments->children[1]);
if (!rhs || !isNativeInteger(rhs->first))
return std::nullopt;
auto uint_val = convertFieldToType(rhs->second, DataTypeUInt64());
return uint_val.safeGet<UInt64>();
};
for (const ConstASTPtr & conjunct : conjuncts)
{
auto field = get_limit_field(conjunct);
if (!field.has_value() || func_name.empty())
{
new_conjuncts.emplace_back(conjunct);
continue;
}
limit = *field;
}
if (new_conjuncts.size() == conjuncts.size())
return node;
if (func_name == "less")
{
if (std::holds_alternative<size_t>(limit))
limit = std::get<size_t>(limit) - 1;
else
return node;
}
auto sorting_step = std::make_shared<SortingStep>(
window_node->getChildren()[0]->getStep()->getOutputStream(),
window_desc.order_by,
limit,
SortingStep::Stage::FULL,
SortDescription{});
auto child_node = SortingNode::createPlanNode(context->nextNodeId(), std::move(sorting_step), PlanNodes{window_node->getChildren()}, node->getStatistics());
UInt64 offset = 0;
auto limit_step = std::make_shared<LimitStep>(child_node->getStep()->getOutputStream(), limit, offset);
child_node = LimitNode::createPlanNode(context->nextNodeId(), std::move(limit_step), PlanNodes{child_node}, node->getStatistics());
if (new_conjuncts.empty())
return child_node;
auto new_filter = PredicateUtils::combineConjuncts(new_conjuncts);
auto new_filter_step = std::make_shared<FilterStep>(child_node->getStep()->getOutputStream(), new_filter);
auto new_filter_node = FilterNode::createPlanNode(context->nextNodeId(), std::move(new_filter_step), {child_node}, node->getStatistics());
return new_filter_node;
}
*/
}

View File

@ -822,7 +822,7 @@ PlanNodePtr PredicateVisitor::visitPartialSortingNode(PartialSortingNode & node,
PlanNodePtr PredicateVisitor::visitSortingNode(SortingNode & node, PredicateContext & predicate_context)
{
if (node.getStep()->getLimit() != 0)
if (node.getStep()->hasPreparedParam() || node.getStep()->getLimitValue() != 0)
return visitPlanNode(node, predicate_context);
return processChild(node, predicate_context);
}

View File

@ -132,7 +132,7 @@ PlanNodePtr PushSortingInfoRewriter::visitAggregatingNode(AggregatingNode & node
{
auto prefix_desc = node.getStep()->getGroupBySortDescription();
SortInfo s{prefix_desc, 0};
SortInfo s{prefix_desc, size_t{0}};
return visitPlanNode(node, s);
}
@ -140,7 +140,7 @@ PlanNodePtr PushSortingInfoRewriter::visitWindowNode(WindowNode & node, SortInfo
{
auto prefix_desc = node.getStep()->getPrefixDescription();
SortInfo s{prefix_desc, 0};
SortInfo s{prefix_desc, size_t{0}};
return visitPlanNode(node, s);
}

View File

@ -2,7 +2,9 @@
#include <Core/SortDescription.h>
#include <Interpreters/Context.h>
#include <Interpreters/prepared_statement.h>
#include <Optimizer/Property/Equivalences.h>
#include <Optimizer/Property/Property.h>
#include <Optimizer/Rewriter/Rewriter.h>
#include <QueryPlan/CTEInfo.h>
#include <QueryPlan/SimplePlanRewriter.h>
@ -49,7 +51,7 @@ private:
struct SortInfo
{
SortDescription sort_desc;
size_t limit;
SizeOrVariable limit;
};
class PushSortingInfoRewriter : public SimplePlanRewriter<SortInfo>

View File

@ -7,7 +7,7 @@ namespace DB::Patterns
PatternBuilder topN()
{
auto result = typeOf(IQueryPlanStep::Type::Sorting);
result.matchingStep<SortingStep>([&](const SortingStep & s) { return s.getLimit() != 0; });
result.matchingStep<SortingStep>([&](const SortingStep & s) { return !s.hasPreparedParam() && s.getLimitValue() != 0; });
return result;
}
PatternBuilder & PatternBuilder::capturedAs(const Capture & capture)

View File

@ -29,7 +29,7 @@ namespace DB
static bool isLimitNeeded(const LimitStep & limit, const PlanNodePtr & node)
{
auto range = PlanNodeCardinality::extractCardinality(*node);
return range.upperBound > limit.getLimit() + limit.getOffset();
return !limit.hasPreparedParam() && range.upperBound > limit.getLimitValue() + limit.getOffsetValue();
}
PatternPtr PushLimitIntoDistinct::getPattern() const
@ -43,13 +43,12 @@ TransformResult PushLimitIntoDistinct::transformImpl(PlanNodePtr node, const Cap
auto distinct = node->getChildren()[0];
const auto * distinct_step = dynamic_cast<const DistinctStep *>(distinct->getStep().get());
// when limit 0, we skip this rule since another rule will delete the whole node
auto limit_value = limit_step->getLimit();
if (limit_value == 0)
if (!isLimitNeeded(*limit_step, distinct))
return {};
if (!isLimitNeeded(*limit_step, distinct))
// when limit 0, we skip this rule since another rule will delete the whole node
auto limit_value = limit_step->getLimitValue();
if (limit_value == 0)
return {};
auto new_distinct = PlanNodeBase::createPlanNode(
@ -57,7 +56,7 @@ TransformResult PushLimitIntoDistinct::transformImpl(PlanNodePtr node, const Cap
std::make_shared<DistinctStep>(
distinct_step->getInputStreams()[0],
distinct_step->getSetSizeLimits(),
limit_step->getLimit() + limit_step->getOffset(),
limit_step->getLimitValue() + limit_step->getOffsetValue(),
distinct_step->getColumns(),
distinct_step->preDistinct()),
distinct->getChildren());
@ -132,8 +131,8 @@ TransformResult PushLimitThroughUnion::transformImpl(PlanNodePtr node, const Cap
context.context->nextNodeId(),
std::make_shared<LimitStep>(
child->getStep()->getOutputStream(),
limit_step->getLimit() + limit_step->getOffset(),
0,
limit_step->getLimitValue() + limit_step->getOffsetValue(),
size_t{0},
limit_step->isAlwaysReadTillEnd(),
limit_step->withTies(),
limit_step->getSortDescription(),
@ -177,8 +176,8 @@ TransformResult PushLimitThroughOuterJoin::transformImpl(PlanNodePtr node, const
context.context->nextNodeId(),
std::make_shared<LimitStep>(
left->getStep()->getOutputStream(),
limit_step->getLimit() + limit_step->getOffset(),
0,
limit_step->getLimitValue() + limit_step->getOffsetValue(),
size_t{0},
limit_step->isAlwaysReadTillEnd(),
limit_step->withTies(),
limit_step->getSortDescription(),
@ -193,8 +192,8 @@ TransformResult PushLimitThroughOuterJoin::transformImpl(PlanNodePtr node, const
context.context->nextNodeId(),
std::make_shared<LimitStep>(
right->getStep()->getOutputStream(),
limit_step->getLimit() + limit_step->getOffset(),
0,
limit_step->getLimitValue() + limit_step->getOffsetValue(),
size_t{0},
limit_step->isAlwaysReadTillEnd(),
limit_step->withTies(),
limit_step->getSortDescription(),
@ -209,7 +208,9 @@ TransformResult PushLimitThroughOuterJoin::transformImpl(PlanNodePtr node, const
PatternPtr LimitZeroToReadNothing::getPattern() const
{
return Patterns::limit().matchingStep<LimitStep>([](const LimitStep & step) { return step.getLimit() == 0; }).result();
return Patterns::limit()
.matchingStep<LimitStep>([](const LimitStep & step) { return !step.hasPreparedParam() && step.getLimitValue() == 0; })
.result();
}
TransformResult LimitZeroToReadNothing::transformImpl(PlanNodePtr node, const Captures &, RuleContext & rule_context)
@ -219,7 +220,9 @@ TransformResult LimitZeroToReadNothing::transformImpl(PlanNodePtr node, const Ca
return {};
const auto & step = *limit_node->getStep();
if (step.getLimit() == 0)
if (step.hasPreparedParam())
return {};
if (step.getLimitValue() == 0)
{
auto read_nothing_step = std::make_shared<ReadNothingStep>(step.getOutputStream().header);
auto read_nothing_node = PlanNodeBase::createPlanNode(rule_context.context->nextNodeId(), std::move(read_nothing_step), {});
@ -248,6 +251,9 @@ TransformResult PushdownLimitIntoWindow::transformImpl(PlanNodePtr node, const C
const auto * limit_step = dynamic_cast<const LimitStep *>(node->getStep().get());
auto source = window->getChildren()[0];
if (limit_step->hasPreparedParam())
return {};
if (source->getStep()->getType() == IQueryPlanStep::Type::Limit || window_step->getWindow().order_by.empty())
{
return {};
@ -257,7 +263,7 @@ TransformResult PushdownLimitIntoWindow::transformImpl(PlanNodePtr node, const C
auto new_sort = PlanNodeBase::createPlanNode(
context.context->nextNodeId(),
std::make_shared<SortingStep>(
source->getStep()->getOutputStream(), window_step->getWindow().order_by, limit_step->getLimit(), SortingStep::Stage::FULL, SortDescription{}),
source->getStep()->getOutputStream(), window_step->getWindow().order_by, limit_step->getLimitValue(), SortingStep::Stage::FULL, SortDescription{}),
{source});
auto new_limit = PlanNodeBase::createPlanNode(
@ -280,9 +286,10 @@ TransformResult PushdownLimitIntoWindow::transformImpl(PlanNodePtr node, const C
PatternPtr PushLimitIntoSorting::getPattern() const
{
return Patterns::limit()
.matchingStep<LimitStep>([](const LimitStep & step) { return step.getLimit() != 0; })
.withSingle(
Patterns::sorting().matchingStep<SortingStep>([](const auto & sorting_step) { return sorting_step.getLimit() == 0; })).result();
.matchingStep<LimitStep>([](const LimitStep & step) { return !step.hasPreparedParam() && step.getLimitValue() != 0; })
.withSingle(Patterns::sorting().matchingStep<SortingStep>(
[](const auto & sorting_step) { return !sorting_step.hasPreparedParam() && sorting_step.getLimitValue() == 0; }))
.result();
}
TransformResult PushLimitIntoSorting::transformImpl(PlanNodePtr node, const Captures &, RuleContext &)
@ -291,8 +298,11 @@ TransformResult PushLimitIntoSorting::transformImpl(PlanNodePtr node, const Capt
auto sorting = node->getChildren()[0];
const auto *sorting_step = dynamic_cast<const SortingStep *>(sorting->getStep().get());
if (limit_step->hasPreparedParam())
return {};
// when limit 0, we skip this rule since another rule will delete the whole node
auto limit_value = limit_step->getLimit();
auto limit_value = limit_step->getLimitValue();
if (limit_value == 0)
return {};
@ -304,7 +314,7 @@ TransformResult PushLimitIntoSorting::transformImpl(PlanNodePtr node, const Capt
std::make_shared<SortingStep>(
sorting_step->getInputStreams()[0],
sorting_step->getSortDescription(),
limit_step->getLimit() + limit_step->getOffset(),
limit_step->getLimitValue() + limit_step->getOffsetValue(),
sorting_step->getStage(),
sorting_step->getPrefixDescription()),
sorting->getChildren());

View File

@ -151,10 +151,13 @@ TransformResult PushLimitIntoTableScan::transformImpl(PlanNodePtr node, const Ca
const auto * limit_step = dynamic_cast<const LimitStep *>(node->getStep().get());
auto table_scan = node->getChildren()[0];
if (limit_step->hasPreparedParam())
return {};
auto copy_table_step = table_scan->getStep()->copy(rule_context.context);
auto * table_step = dynamic_cast<TableScanStep *>(copy_table_step.get());
bool applied = table_step->setLimit(limit_step->getLimit() + limit_step->getOffset(), rule_context.context);
auto table_step = dynamic_cast<TableScanStep *>(copy_table_step.get());
bool applied = table_step->setLimit(limit_step->getLimitValue() + limit_step->getOffsetValue(), rule_context.context);
if (!applied)
return {}; // repeat calls

View File

@ -349,7 +349,7 @@ TransformResult PushPartialSortingThroughExchange::transformImpl(PlanNodePtr nod
static bool isLimitNeeded(const LimitStep & limit, const PlanNodePtr & node)
{
auto range = PlanNodeCardinality::extractCardinality(*node);
return range.upperBound > limit.getLimit() + limit.getOffset();
return !limit.hasPreparedParam() && range.upperBound > limit.getLimitValue() + limit.getOffsetValue();
}
PatternPtr PushPartialLimitThroughExchange::getPattern() const
@ -372,8 +372,8 @@ TransformResult PushPartialLimitThroughExchange::transformImpl(PlanNodePtr node,
{
auto partial_limit = std::make_unique<LimitStep>(
exchange_child->getStep()->getOutputStream(),
step->getLimit() + step->getOffset(),
0,
step->getLimitValue() + step->getOffsetValue(),
size_t{0},
false,
false,
step->getSortDescription(),

View File

@ -295,7 +295,7 @@ PatternPtr RemoveRedundantLimit::getPattern() const
TransformResult RemoveRedundantLimit::transformImpl(PlanNodePtr node, const Captures &, RuleContext & context)
{
auto * limit_node = dynamic_cast<LimitNode *>(node.get());
if (limit_node->getStep()->getLimit() == 0)
if (!limit_node->getStep()->hasPreparedParam() && limit_node->getStep()->getLimitValue() == 0)
{
auto null_step = std::make_unique<ReadNothingStep>(limit_node->getStep()->getOutputStream().header);
auto null_node = PlanNodeBase::createPlanNode(context.context->nextNodeId(), std::move(null_step));

View File

@ -36,9 +36,11 @@ TransformResult CreateTopNFilteringForAggregating::transformImpl(PlanNodePtr nod
{
const auto & topn_step = dynamic_cast<const SortingStep &>(*node->getStep());
if (topn_step.getLimit() > getMaxRowsToUseTopnFiltering(context.context))
if (topn_step.getLimitValue() > getMaxRowsToUseTopnFiltering(context.context))
return {};
if (topn_step.hasPreparedParam())
return {};
auto & aggregate_node = node->getChildren()[0];
const auto & aggregate_step = dynamic_cast<const AggregatingStep &>(*aggregate_node->getStep());
@ -63,7 +65,10 @@ TransformResult CreateTopNFilteringForAggregating::transformImpl(PlanNodePtr nod
const auto & aggregate_child_node = aggregate_node->getChildren()[0];
auto topn_filter_step = std::make_shared<TopNFilteringStep>(
aggregate_child_node->getStep()->getOutputStream(), topn_step.getSortDescription(), topn_step.getLimit(), TopNModel::DENSE_RANK);
aggregate_child_node->getStep()->getOutputStream(),
topn_step.getSortDescription(),
topn_step.getLimitValue(),
TopNModel::DENSE_RANK);
auto topn_filter_node = PlanNodeBase::createPlanNode(context.context->nextNodeId(), topn_filter_step, PlanNodes{aggregate_child_node});
aggregate_node->replaceChildren(PlanNodes{topn_filter_node});

View File

@ -45,6 +45,10 @@ class UnwrapCastInComparisonVisitor: public ASTVisitor<ASTPtr, UnwrapCastInCompa
ASTPtr visitASTLiteral(ASTPtr & node, UnwrapCastInComparisonContext &) override { return node; }
ASTPtr visitASTIdentifier(ASTPtr & node, UnwrapCastInComparisonContext &) override { return node; }
ASTPtr visitASTSubquery(ASTPtr & node, UnwrapCastInComparisonContext &) override { return node; }
ASTPtr visitASTPreparedParameter(ASTPtr & node, UnwrapCastInComparisonContext &) override
{
return node;
}
ASTPtr rewriteArgs(ASTFunction & function, UnwrapCastInComparisonContext & context, bool first_only = false);
static bool isComparisonFunction(const ASTFunction & function);

View File

@ -0,0 +1,50 @@
#include <Parsers/ASTPreparedParameter.h>
#include <IO/Operators.h>
#include <IO/WriteHelpers.h>
#include <Common/Exception.h>
#include <Common/quoteString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
};
void ASTPreparedParameter::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_substitution : "") << '[' << (settings.hilite ? hilite_identifier : "")
<< backQuoteIfNeed(name) << (settings.hilite ? hilite_substitution : "") << ':'
<< (settings.hilite ? hilite_function : "") << type << (settings.hilite ? hilite_substitution : "") << ']'
<< (settings.hilite ? hilite_none : "");
}
void ASTPreparedParameter::appendColumnNameImpl(WriteBuffer & ostr) const
{
writeString("PreparedParam_" + name, ostr);
}
void ASTPreparedParameter::serialize(WriteBuffer & buf) const
{
ASTWithAlias::serialize(buf);
writeBinary(name, buf);
writeBinary(type, buf);
}
ASTPtr ASTPreparedParameter::deserialize(ReadBuffer & buf)
{
auto prepared_param = std::make_shared<ASTPreparedParameter>();
prepared_param->deserializeImpl(buf);
return prepared_param;
}
void ASTPreparedParameter::deserializeImpl(ReadBuffer & buf)
{
ASTWithAlias::deserializeImpl(buf);
readBinary(name, buf);
readBinary(type, buf);
}
}

View File

@ -0,0 +1,49 @@
#pragma once
#include <Parsers/ASTWithAlias.h>
namespace DB
{
/// Parameter in prepared query with name and type of substitution ([name:type]).
/// Example: PREPARE SELECT * FROM table WHERE id = [pid:UInt16].
class ASTPreparedParameter : public ASTWithAlias
{
public:
String name;
String type;
ASTPreparedParameter() = default;
ASTPreparedParameter(const String & name_, const String & type_) : name(name_), type(type_)
{
}
/** Get the text that identifies this element. */
String getID(char delim) const override
{
return String("PreparedParameter") + delim + name;
}
ASTType getType() const override
{
return ASTType::ASTPreparedParameter;
}
ASTPtr clone() const override
{
return std::make_shared<ASTPreparedParameter>(*this);
}
void serialize(WriteBuffer & buf) const override;
static ASTPtr deserialize(ReadBuffer & buf);
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;
void appendColumnNameImpl(WriteBuffer & ostr) const override;
private:
void deserializeImpl(ReadBuffer & buf) override;
};
}

View File

@ -0,0 +1,106 @@
#include <Parsers/ASTPreparedStatement.h>
namespace DB
{
ASTPtr ASTCreatePreparedStatementQuery::clone() const
{
auto res = std::make_shared<ASTCreatePreparedStatementQuery>(*this);
res->query = query->clone();
res->children.clear();
res->children.push_back(res->query);
return res;
}
void ASTCreatePreparedStatementQuery::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE " << (settings.hilite ? hilite_none : "");
if (is_permanent)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "PERMANENT " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << "PREPARED STATEMENT " << (settings.hilite ? hilite_none : "");
if (if_not_exists)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "IF NOT EXISTS " << (settings.hilite ? hilite_none : "");
else if (or_replace)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "OR REPLACE " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_identifier : "") << name << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << (settings.hilite ? hilite_none : "");
if (getQuery())
{
settings.ostr << settings.nl_or_ws;
getQuery()->formatImpl(settings, state, frame);
}
}
ASTPtr ASTExecutePreparedStatementQuery::clone() const
{
auto res = std::make_shared<ASTExecutePreparedStatementQuery>(*this);
if (values)
res->values = values->clone();
cloneOutputOptions(*res);
return res;
}
void ASTExecutePreparedStatementQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXECUTE PREPARED STATEMENT " << (settings.hilite ? hilite_identifier : "")
<< name << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << " USING" << (settings.hilite ? hilite_none : "");
if (getValues())
{
settings.ostr << settings.nl_or_ws;
getValues()->formatImpl(settings, state, frame);
}
}
ASTPtr ASTShowPreparedStatementQuery::clone() const
{
auto res = std::make_shared<ASTShowPreparedStatementQuery>(*this);
return res;
}
void ASTShowPreparedStatementQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
if (show_explain)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "EXPLAIN " << (settings.hilite ? hilite_none : "");
else
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (settings.hilite ? hilite_none : "");
if (show_create)
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE " << (settings.hilite ? hilite_none : "");
if (show_explain || show_create)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "PREPARED STATEMENT " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_identifier : "") << name << (settings.hilite ? hilite_none : "");
}
else
settings.ostr << (settings.hilite ? hilite_keyword : "") << "PREPARED STATEMENTS" << (settings.hilite ? hilite_none : "");
}
ASTPtr ASTDropPreparedStatementQuery::clone() const
{
auto res = std::make_shared<ASTDropPreparedStatementQuery>(*this);
return res;
}
void ASTDropPreparedStatementQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP PREPARED STATEMENT ";
if (if_exists)
settings.ostr << "IF EXISTS ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << name << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
}
}

View File

@ -0,0 +1,138 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/IAST.h>
namespace DB
{
class ASTCreatePreparedStatementQuery : public IAST, public ASTQueryWithOnCluster
{
public:
String name;
ASTPtr query;
bool if_not_exists = false;
bool or_replace = false;
bool is_permanent = false;
/** Get the text that identifies this element. */
String getID(char) const override
{
return "CreatePreparedStatement";
}
ASTType getType() const override
{
return ASTType::ASTCreatePreparedStatementQuery;
}
ASTPtr clone() const override;
String getName() const
{
return name;
}
ASTPtr getQuery() const
{
return query;
}
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override
{
return removeOnCluster<ASTCreatePreparedStatementQuery>(clone());
}
protected:
void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
};
class ASTExecutePreparedStatementQuery : public ASTQueryWithOutput
{
public:
String name;
ASTPtr values;
/** Get the text that identifies this element. */
String getID(char) const override
{
return "ExecutePreparedStatement";
}
ASTType getType() const override
{
return ASTType::ASTExecutePreparedStatementQuery;
}
ASTPtr clone() const override;
const String & getName() const
{
return name;
}
ASTPtr getValues() const
{
return values;
}
protected:
void formatQueryImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
};
class ASTShowPreparedStatementQuery : public ASTQueryWithOutput
{
public:
String name;
bool show_create = false;
bool show_explain = false;
/** Get the text that identifies this element. */
String getID(char) const override
{
return "ShowPreparedStatement";
}
ASTType getType() const override
{
return ASTType::ASTShowPreparedStatementQuery;
}
ASTPtr clone() const override;
protected:
void formatQueryImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
};
class ASTDropPreparedStatementQuery : public IAST, public ASTQueryWithOnCluster
{
public:
String name;
bool if_exists = false;
/** Get the text that identifies this element. */
String getID(char) const override
{
return "DropPreparedStatement";
}
ASTType getType() const override
{
return ASTType::ASTDropPreparedStatementQuery;
}
ASTPtr clone() const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override
{
return removeOnCluster<ASTDropPreparedStatementQuery>(clone());
}
protected:
void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override;
};
}

View File

@ -54,6 +54,8 @@
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTPartToolKit.h>
#include <Parsers/ASTPartition.h>
#include <Parsers/ASTPreparedParameter.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTProjectionSelectQuery.h>
#include <Parsers/ASTQualifiedAsterisk.h>

View File

@ -38,6 +38,7 @@
#include <Parsers/ASTOptimizeQuery.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTPartition.h>
#include <Parsers/ASTPreparedParameter.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/ASTQuantifiedComparison.h>
#include <Parsers/ASTRenameQuery.h>

View File

@ -55,11 +55,12 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTAssignment.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCase.h>
#include <Parsers/ParserPreparedParameter.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/parseIntervalKind.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserCase.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserCreateQuery.h>
@ -2585,7 +2586,8 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|| ParserExistsExpression(dt).parse(pos, node, expected)
|| ParserCompoundIdentifier(false, true).parse(pos, node, expected)
|| ParserSubstitution(dt).parse(pos, node, expected)
|| ParserMySQLGlobalVariable().parse(pos, node, expected);
|| ParserMySQLGlobalVariable().parse(pos, node, expected)
|| ParserPreparedParameter().parse(pos, node, expected);
}

View File

@ -154,7 +154,12 @@ class ReadBuffer;
M(ASTQuantifiedComparison) \
M(ASTTableColumnReference) \
M(ASTUpdateQuery) \
M(ASTBitEngineConstraintDeclaration)
M(ASTBitEngineConstraintDeclaration)\
M(ASTPreparedParameter) \
M(ASTCreatePreparedStatementQuery) \
M(ASTExecutePreparedStatementQuery) \
M(ASTShowPreparedStatementQuery) \
M(ASTDropPreparedStatementQuery)
#define ENUM_TYPE(ITEM) ITEM,
enum class ASTType : UInt8

View File

@ -0,0 +1,43 @@
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
#include <Parsers/ASTDataType.h>
#include <Parsers/ASTIdentifier_fwd.h>
#include <Parsers/ASTPreparedParameter.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserDataType.h>
#include <Parsers/ParserPreparedParameter.h>
namespace DB
{
bool ParserPreparedParameter::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserToken(TokenType::OpeningSquareBracket).ignore(pos, expected))
return false;
ASTPtr type_node;
auto prepared_parameter = std::make_shared<ASTPreparedParameter>();
ParserIdentifier name_p;
ASTPtr identifier;
if (!name_p.parse(pos, identifier, expected))
return false;
if (!ParserToken(TokenType::Colon).ignore(pos, expected))
return false;
if (!name_p.parse(pos, type_node, expected))
return false;
if (!ParserToken(TokenType::ClosingSquareBracket).ignore(pos, expected))
return false;
tryGetIdentifierNameInto(identifier, prepared_parameter->name);
tryGetIdentifierNameInto(type_node, prepared_parameter->type);
node = std::move(prepared_parameter);
return true;
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/IParser.h>
#include <Parsers/IParserBase.h>
namespace DB
{
class ParserPreparedParameter : public IParserDialectBase
{
protected:
const char * getName() const override
{
return "PreparedParameter";
}
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
public:
using IParserDialectBase::IParserDialectBase;
};
}

View File

@ -0,0 +1,193 @@
#include <IO/ReadBufferFromMemory.h>
#include <IO/ReadHelpers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ParserPreparedStatement.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
namespace DB
{
bool ParserCreatePreparedStatementQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_permanent("PERMANENT");
ParserKeyword s_prepared_statement("PREPARED STATEMENT");
ParserKeyword s_if_not_exists("IF NOT EXISTS");
ParserKeyword s_or_replace("OR REPLACE");
ParserKeyword s_on("ON");
ParserKeyword s_as("AS");
bool if_not_exists = false;
bool or_replace = false;
bool is_permanent = false;
if (!s_create.ignore(pos, expected))
return false;
if (s_permanent.ignore(pos, expected))
is_permanent = true;
if (!s_prepared_statement.ignore(pos, expected))
return false;
if (s_if_not_exists.ignore(pos, expected))
if_not_exists = true;
else if (s_or_replace.ignore(pos, expected))
or_replace = true;
ParserCompoundIdentifier name_p;
ASTPtr identifier;
if (!name_p.parse(pos, identifier, expected))
return false;
String cluster_str;
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
if (!s_as.parse(pos, identifier, expected))
return false;
ASTPtr query;
ParserSelectWithUnionQuery select_p(dt);
if (!select_p.parse(pos, query, expected))
return false;
auto prepare = std::make_shared<ASTCreatePreparedStatementQuery>();
tryGetIdentifierNameInto(identifier, prepare->name);
prepare->cluster = std::move(cluster_str);
prepare->is_permanent = is_permanent;
prepare->if_not_exists = if_not_exists;
prepare->or_replace = or_replace;
prepare->query = query;
prepare->children.push_back(prepare->query);
node = prepare;
return true;
}
bool ParserExecutePreparedStatementQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_execute("EXECUTE PREPARED STATEMENT");
ParserKeyword s_using("USING");
if (!s_execute.ignore(pos, expected))
return false;
ParserCompoundIdentifier name_p;
ASTPtr identifier;
if (!name_p.parse(pos, identifier, expected))
return false;
ASTPtr settings;
if (s_using.ignore(pos, expected))
{
ParserSetQuery parser_settings(true);
if (!parser_settings.parse(pos, settings, expected))
return false;
}
else
{
settings = std::make_shared<ASTSetQuery>();
}
auto execute = std::make_shared<ASTExecutePreparedStatementQuery>();
tryGetIdentifierNameInto(identifier, execute->name);
execute->values = settings;
node = execute;
return true;
}
bool ParserShowPreparedStatementQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_show("SHOW");
ParserKeyword s_create("CREATE");
ParserKeyword s_explain("EXPLAIN");
ParserKeyword s_prepared_statement("PREPARED STATEMENT");
ParserKeyword s_prepared_statements("PREPARED STATEMENTS");
ParserCompoundIdentifier name_p;
ASTPtr identifier;
bool create = false;
bool explain = false;
if (s_show.ignore(pos, expected))
{
if (s_create.ignore(pos, expected))
{
create = true;
if (!s_prepared_statement.ignore(pos, expected))
return false;
if (!name_p.parse(pos, identifier, expected))
return false;
}
else if (s_prepared_statements.ignore(pos, expected))
{
}
else
return false;
}
else if (s_explain.ignore(pos, expected))
{
explain = true;
if (!s_prepared_statement.ignore(pos, expected))
return false;
if (!name_p.parse(pos, identifier, expected))
return false;
}
else
return false;
auto show_prepare = std::make_shared<ASTShowPreparedStatementQuery>();
if (identifier)
tryGetIdentifierNameInto(identifier, show_prepare->name);
show_prepare->show_create = create;
show_prepare->show_explain = explain;
node = show_prepare;
return true;
}
bool ParserDropPreparedStatementQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_if_exists("IF EXISTS");
ParserKeyword s_on("ON");
ParserKeyword s_prepare("PREPARED STATEMENT");
if (!s_drop.ignore(pos, expected) || !s_prepare.ignore(pos, expected))
return false;
bool if_exists = false;
if (s_if_exists.ignore(pos, expected))
if_exists = true;
ParserCompoundIdentifier name_p;
ASTPtr identifier;
if (!name_p.parse(pos, identifier, expected))
return false;
String cluster_str;
if (s_on.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
auto drop = std::make_shared<ASTDropPreparedStatementQuery>();
tryGetIdentifierNameInto(identifier, drop->name);
drop->cluster = std::move(cluster_str);
drop->if_exists = if_exists;
node = drop;
return true;
}
}

View File

@ -0,0 +1,66 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/IParser.h>
#include <Parsers/IParserBase.h>
namespace DB
{
/**
* Create Prepared Statement
*/
class ParserCreatePreparedStatementQuery : public IParserDialectBase
{
public:
explicit ParserCreatePreparedStatementQuery(const ParserSettingsImpl & parser_settings_impl_ = ParserSettings::CLICKHOUSE)
: IParserDialectBase(parser_settings_impl_)
{
}
protected:
const char * getName() const override
{
return "CreatePreparedStatement";
}
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/**
* Execute Prepared Statement
*/
class ParserExecutePreparedStatementQuery : public IParserBase
{
protected:
const char * getName() const override
{
return "ExecutePreparedStatement";
}
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/**
* Show Prepared Statement
*/
class ParserShowPreparedStatementQuery : public IParserBase
{
protected:
const char * getName() const override
{
return "ShowPreparedStatement";
}
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/**
* Drop Prepared Statement
*/
class ParserDropPreparedStatementQuery : public IParserBase
{
protected:
const char * getName() const override
{
return "DropPreparedStatement";
}
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -19,6 +19,7 @@
* All Bytedance's Modifications are Copyright (2023) Bytedance Ltd. and/or its affiliates.
*/
#include <Parsers/ParserAdviseQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserAlterWarehouseQuery.h>
#include <Parsers/ParserCreateQuery.h>
@ -38,8 +39,8 @@
#include <Parsers/ParserGrantQuery.h>
#include <Parsers/ParserInsertQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserPreparedStatement.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/ParserAdviseQuery.h>
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/ParserRenameQuery.h>
#include <Parsers/ParserSetQuery.h>
@ -82,36 +83,25 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserDeleteQuery delete_p;
ParserUpdateQuery update_query_p;
ParserCreateBinding create_binding(dt);
ParserCreatePreparedStatementQuery prepare(dt);
ParserDropPreparedStatementQuery drop_prepare;
ParserShowBindings show_bindings;
ParserDropBinding drop_binding(dt);
ParserAlterQuery alter_p(dt);
bool res = query_with_output_p.parse(pos, node, expected)
|| insert_p.parse(pos, node, expected)
|| use_p.parse(pos, node, expected)
|| switch_p.parse(pos,node,expected)
|| set_role_p.parse(pos, node, expected)
|| set_p.parse(pos, node, expected)
|| advise_p.parse(pos, node, expected)
|| system_p.parse(pos, node, expected)
|| create_user_p.parse(pos, node, expected)
|| create_role_p.parse(pos, node, expected)
|| create_quota_p.parse(pos, node, expected)
|| create_row_policy_p.parse(pos, node, expected)
|| create_settings_profile_p.parse(pos, node, expected)
|| drop_access_entity_p.parse(pos, node, expected)
|| grant_p.parse(pos, node, expected)
|| external_ddl_p.parse(pos, node, expected)
|| create_warehouse_p.parse(pos, node, expected)
|| alter_warehouse_p.parse(pos, node, expected)
|| drop_warehouse_p.parse(pos, node, expected)
|| show_warehouse_p.parse(pos, node, expected)
|| create_worker_group_p.parse(pos, node, expected)
|| drop_worker_group_p.parse(pos, node, expected)
|| delete_p.parse(pos, node, expected)
|| update_query_p.parse(pos, node ,expected)
|| create_binding.parse(pos, node, expected)
|| show_bindings.parse(pos, node, expected)
|| drop_binding.parse(pos, node, expected);
bool res = query_with_output_p.parse(pos, node, expected) || insert_p.parse(pos, node, expected) || use_p.parse(pos, node, expected)
|| switch_p.parse(pos, node, expected) || set_role_p.parse(pos, node, expected) || set_p.parse(pos, node, expected)
|| advise_p.parse(pos, node, expected) || system_p.parse(pos, node, expected) || create_user_p.parse(pos, node, expected)
|| create_role_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected)
|| create_row_policy_p.parse(pos, node, expected) || create_settings_profile_p.parse(pos, node, expected)
|| drop_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected)
|| external_ddl_p.parse(pos, node, expected) || create_warehouse_p.parse(pos, node, expected)
|| alter_warehouse_p.parse(pos, node, expected) || drop_warehouse_p.parse(pos, node, expected)
|| show_warehouse_p.parse(pos, node, expected) || create_worker_group_p.parse(pos, node, expected)
|| drop_worker_group_p.parse(pos, node, expected) || delete_p.parse(pos, node, expected)
|| update_query_p.parse(pos, node, expected) || create_binding.parse(pos, node, expected)
|| show_bindings.parse(pos, node, expected) || drop_binding.parse(pos, node, expected) || alter_p.parse(pos, node, expected)
|| prepare.parse(pos, node, expected) || drop_prepare.parse(pos, node, expected);
return res;
}

View File

@ -38,13 +38,33 @@
#include <Parsers/ASTExplainQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ParserCheckQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserDescribeTableQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserDumpQuery.h>
#include <Parsers/ParserExplainQuery.h>
#include <Parsers/ParserKillQueryQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
#include <Parsers/ParserPreparedStatement.h>
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/ParserRefreshQuery.h>
#include <Parsers/ParserRenameQuery.h>
#include <Parsers/ParserReproduceQuery.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ParserShowAccessEntitiesQuery.h>
#include <Parsers/ParserShowAccessQuery.h>
#include <Parsers/ParserShowColumnsQuery.h>
#include <Parsers/ParserShowCreateAccessEntityQuery.h>
#include <Parsers/ParserShowGrantsQuery.h>
#include <Parsers/ParserShowPrivilegesQuery.h>
#include <Parsers/ParserExplainQuery.h>
#include <Parsers/ParserShowProcesslistQuery.h>
#include <Parsers/ParserShowTablesQuery.h>
#include <Parsers/ParserStatsQuery.h>
#include <Parsers/ParserTablePropertiesQuery.h>
#include <Parsers/ParserWatchQuery.h>
#include <Parsers/QueryWithOutputSettingsPushDownVisitor.h>
#include <Parsers/ParserRefreshQuery.h>
#include <Parsers/ParserStatsQuery.h>
@ -93,16 +113,19 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ParserBeginQuery begin_p;
ParserCommitQuery commit_p;
ParserRollbackQuery rollback_p;
ParserExecutePreparedStatementQuery execute_p;
ParserShowPreparedStatementQuery show_prepared;
ASTPtr query;
bool parsed =
explain_p.parse(pos, query, expected)
bool parsed = execute_p.parse(pos, query, expected)
|| select_p.parse(pos, query, expected)
|| explain_p.parse(pos, query, expected)
|| reproduce_p.parse(pos, query, expected)
|| dump_p.parse(pos, query, expected)
|| select_p.parse(pos, query, expected)
|| show_create_access_entity_p.parse(pos, query, expected) /// should be before `show_tables_p`
|| show_tables_p.parse(pos, query, expected)
|| show_prepared.parse(pos, query, expected)
|| show_setting_p.parse(pos, query, expected)
|| show_columns_p.parse(pos, query, expected)
|| table_p.parse(pos, query, expected)

View File

@ -227,7 +227,7 @@ QueryPlanStepPtr createMergeSortingStep()
QueryPlanStepPtr createLimitStep()
{
DataStream stream = createDataStream();
return std::make_unique<LimitStep>(stream, 0, 0);
return std::make_unique<LimitStep>(stream, UInt64(0), UInt64(0));
}
QueryPlanStepPtr createLimitByStep()

View File

@ -0,0 +1,22 @@
#include <Protos/PreparedStatementHelper.h>
namespace DB
{
void setSizeOrVariableToProto(const SizeOrVariable & size_or_var, Protos::SizeOrVariable & proto)
{
if (const auto * size = std::get_if<size_t>(&size_or_var))
proto.set_size(*size);
else
proto.set_variable(std::get<String>(size_or_var));
}
std::optional<SizeOrVariable> getSizeOrVariableFromProto(const Protos::SizeOrVariable & proto)
{
if (proto.has_size())
return SizeOrVariable{proto.size()};
else if (proto.has_variable())
return SizeOrVariable{proto.variable()};
else
return std::nullopt;
}
}

View File

@ -0,0 +1,11 @@
#pragma once
#include <Interpreters/prepared_statement.h>
#include <Protos/plan_node_utils.pb.h>
#include <optional>
namespace DB
{
void setSizeOrVariableToProto(const SizeOrVariable & size_or_var, Protos::SizeOrVariable & proto);
std::optional<SizeOrVariable> getSizeOrVariableFromProto(const Protos::SizeOrVariable & proto);
}

View File

@ -55,12 +55,14 @@ message LimitByStep {
message LimitStep {
required ITransformingStep query_plan_base = 1;
required uint64 limit = 2;
required uint64 offset = 3;
required uint64 limit = 2; // deprecated, use limit_or_var instead
required uint64 offset = 3; // deprecated, use offset_or_var instead
required bool always_read_till_end = 4;
required bool with_ties = 5;
repeated SortColumnDescription description = 6;
required bool partial = 7;
optional SizeOrVariable limit_or_var = 8;
optional SizeOrVariable offset_or_var = 9;
}
message MarkDistinctStep {
@ -92,10 +94,11 @@ message SortingStep {
required ITransformingStep query_plan_base = 1;
repeated SortColumnDescription result_description = 2;
required uint64 limit = 3;
required uint64 limit = 3; // deprecated, use limit_or_var instead
required bool partial = 4;
repeated SortColumnDescription prefix_description = 5;
optional Stage.Enum stage = 6;
optional SizeOrVariable limit_or_var = 7;
}
message ValuesStep {
@ -438,3 +441,8 @@ message QueryPlan {
map<uint64, uint64> cte_id_mapping = 3;
required PlanMode.Enum mode = 4;
}
message PreparedStatement {
required string query = 1;
}

View File

@ -4,6 +4,13 @@ import "cnch_common.proto";
import "enum.proto";
package DB.Protos;
message SizeOrVariable {
oneof size_or_variable {
uint64 size = 1;
string variable = 2;
}
}
message Field {
required FieldType.Enum type = 1;
optional bytes blob = 2;
@ -107,7 +114,7 @@ message Partitioning {
// possibly nullptr
message AST {
// empty bytes means nullptr
required bytes blob = 1; // exact binary
required bytes blob = 1; // exact binary
optional string text = 2; // only for read
// TODO: use better serialize method
}
@ -269,7 +276,7 @@ message WindowFrame {
required bool begin_preceding = 5;
required BoundaryType.Enum end_type = 6;
required Field end_offset = 7;
required bool end_preceding = 8;
required bool end_preceding = 8;
}
message WindowFunctionDescription {
@ -300,3 +307,11 @@ message Graph {
required AST filter = 3;
}
message SettingChange {
required string name = 1;
required Field value = 2;
}
message SettingsChanges {
repeated SettingChange settings_changes = 1;
}

View File

@ -0,0 +1,12 @@
#include <QueryPlan/Assignment.h>
namespace DB
{
Assignments Assignments::copy() const
{
Assignments copied;
for (const auto & [symbol, ast] : *this)
copied.emplace(symbol, ast->clone());
return copied;
}
}

View File

@ -24,6 +24,10 @@ namespace DB
{
using Assignment = std::pair<String, ConstASTPtr>;
// using Assignments = std::vector<Assignment>;
using Assignments = LinkedHashMap<String, ConstASTPtr>;
class Assignments : public LinkedHashMap<String, ConstASTPtr>
{
public:
using LinkedHashMap::LinkedHashMap;
Assignments copy() const;
};
}

View File

@ -175,7 +175,7 @@ void FilterStep::toProto(Protos::FilterStep & proto, bool) const
std::shared_ptr<IQueryPlanStep> FilterStep::copy(ContextPtr) const
{
return std::make_shared<FilterStep>(input_streams[0], filter, remove_filter_column);
return std::make_shared<FilterStep>(input_streams[0], filter->clone(), remove_filter_column);
}
ConstASTPtr
@ -197,4 +197,8 @@ FilterStep::rewriteRuntimeFilter(const ConstASTPtr & filter, QueryPipeline & /*p
return PredicateUtils::combineConjuncts(predicates);
}
void FilterStep::prepare(const PreparedStatementContext & prepared_context)
{
prepared_context.prepare(filter);
}
}

View File

@ -59,6 +59,8 @@ public:
static ConstASTPtr rewriteRuntimeFilter(const ConstASTPtr & filter, QueryPipeline & pipeline, const BuildQueryPipelineSettings & build_context);
void prepare(const PreparedStatementContext & prepared_context) override;
private:
ActionsDAGPtr actions_dag;
ConstASTPtr filter;

View File

@ -1816,15 +1816,19 @@ String StepPrinter::printTableScanStep(const TableScanStep & step)
if (step.getQueryInfo().input_order_info)
{
const auto & prefix_descs = step.getQueryInfo().input_order_info->order_key_prefix_descr;
const auto & input_order_info = step.getQueryInfo().input_order_info;
details << "Input Order Info: \\n";
const auto & prefix_descs = input_order_info->order_key_prefix_descr;
if (!prefix_descs.empty())
{
details << "prefix desc";
details << "prefix desc: \\n";
for (const auto & desc : prefix_descs)
{
details << desc.column_name << " " << desc.direction << " " << desc.nulls_direction << "\\n";
}
}
details << "direction: " << input_order_info->direction << "\\n";
details << "|";
}
@ -1963,11 +1967,8 @@ String StepPrinter::printValuesStep(const ValuesStep & step)
String StepPrinter::printLimitStep(const LimitStep & step)
{
std::stringstream details;
auto limit = step.getLimit();
auto offset = step.getOffset();
details << "Limit:" << limit << "|";
details << "Offset:" << offset;
details << "|";
std::visit([&](const auto & v) { details << "Limit:" << v << "|"; }, step.getLimit());
std::visit([&](const auto & v) { details << "Offset:" << v << "|"; }, step.getOffset());
details << "Output\\n";
for (const auto & column : step.getOutputStream().header)
{
@ -2054,7 +2055,7 @@ String StepPrinter::printSortingStep(const SortingStep & step)
}
}
details << "|";
details << "Limit: " << step.getLimit();
details << "Limit: " << step.getLimitValue();
if (step.getStage() == SortingStep::Stage::FULL)
{
details << "|";

View File

@ -16,6 +16,7 @@
#pragma once
#include <Core/Block.h>
#include <Core/SortDescription.h>
#include <Interpreters/prepared_statement.h>
#include <Parsers/IAST_fwd.h>
#include <Protos/EnumMacros.h>
#include <Protos/plan_node.pb.h>
@ -288,6 +289,9 @@ public:
}
static String toString(Type type);
virtual void prepare(const PreparedStatementContext &)
{
}
protected:
DataStreams input_streams;
std::optional<DataStream> output_stream;

View File

@ -13,10 +13,11 @@
* limitations under the License.
*/
#include <QueryPlan/LimitStep.h>
#include <Processors/QueryPipeline.h>
#include <Processors/LimitTransform.h>
#include <IO/Operators.h>
#include <Processors/LimitTransform.h>
#include <Processors/QueryPipeline.h>
#include <Protos/PreparedStatementHelper.h>
#include <QueryPlan/LimitStep.h>
#include <Common/JSONBuilder.h>
namespace DB
@ -40,15 +41,18 @@ static ITransformingStep::Traits getTraits()
LimitStep::LimitStep(
const DataStream & input_stream_,
size_t limit_, size_t offset_,
SizeOrVariable limit_,
SizeOrVariable offset_,
bool always_read_till_end_,
bool with_ties_,
SortDescription description_,
bool partial_)
: ITransformingStep(input_stream_, input_stream_.header, getTraits())
, limit(limit_), offset(offset_)
, limit(limit_)
, offset(offset_)
, always_read_till_end(always_read_till_end_)
, with_ties(with_ties_), description(std::move(description_))
, with_ties(with_ties_)
, description(std::move(description_))
, partial(partial_)
{
}
@ -69,7 +73,7 @@ void LimitStep::updateInputStream(DataStream input_stream_)
void LimitStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &)
{
auto transform = std::make_shared<LimitTransform>(
pipeline.getHeader(), limit, offset, pipeline.getNumStreams(), always_read_till_end, with_ties, description);
pipeline.getHeader(), getLimitValue(), getOffsetValue(), pipeline.getNumStreams(), always_read_till_end, with_ties, description);
pipeline.addTransform(std::move(transform));
}
@ -77,8 +81,8 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipe
void LimitStep::describeActions(FormatSettings & settings) const
{
String prefix(settings.offset, ' ');
settings.out << prefix << "Limit " << limit << '\n';
settings.out << prefix << "Offset " << offset << '\n';
std::visit([&](const auto & x) { settings.out << prefix << "Limit " << x << '\n'; }, limit);
std::visit([&](const auto & x) { settings.out << prefix << "Offset " << x << '\n'; }, offset);
if (with_ties || always_read_till_end)
{
@ -102,8 +106,8 @@ void LimitStep::describeActions(FormatSettings & settings) const
void LimitStep::describeActions(JSONBuilder::JSONMap & map) const
{
map.add("Limit", limit);
map.add("Offset", offset);
std::visit([&](const auto & x) { map.add("Limit", x); }, limit);
std::visit([&](const auto & x) { map.add("Offset", x); }, offset);
map.add("With Ties", with_ties);
map.add("Reads All Data", always_read_till_end);
}
@ -111,8 +115,10 @@ void LimitStep::describeActions(JSONBuilder::JSONMap & map) const
void LimitStep::toProto(Protos::LimitStep & proto, bool) const
{
ITransformingStep::serializeToProtoBase(*proto.mutable_query_plan_base());
proto.set_limit(limit);
proto.set_offset(offset);
proto.set_limit(0);
proto.set_offset(0);
setSizeOrVariableToProto(limit, *proto.mutable_limit_or_var());
setSizeOrVariableToProto(offset, *proto.mutable_offset_or_var());
proto.set_always_read_till_end(always_read_till_end);
proto.set_with_ties(with_ties);
for (const auto & element : description)
@ -125,6 +131,8 @@ std::shared_ptr<LimitStep> LimitStep::fromProto(const Protos::LimitStep & proto,
auto [step_description, base_input_stream] = ITransformingStep::deserializeFromProtoBase(proto.query_plan_base());
auto limit = proto.limit();
auto offset = proto.offset();
auto limit_or_var = getSizeOrVariableFromProto(proto.limit_or_var());
auto offset_or_var = getSizeOrVariableFromProto(proto.offset_or_var());
auto always_read_till_end = proto.always_read_till_end();
auto with_ties = proto.with_ties();
SortDescription description;
@ -135,7 +143,14 @@ std::shared_ptr<LimitStep> LimitStep::fromProto(const Protos::LimitStep & proto,
description.emplace_back(std::move(element));
}
auto partial = proto.partial();
auto step = std::make_shared<LimitStep>(base_input_stream, limit, offset, always_read_till_end, with_ties, description, partial);
auto step = std::make_shared<LimitStep>(
base_input_stream,
limit_or_var ? *limit_or_var : limit,
offset_or_var ? *offset_or_var : offset,
always_read_till_end,
with_ties,
description,
partial);
step->setStepDescription(step_description);
return step;
}
@ -144,4 +159,10 @@ std::shared_ptr<IQueryPlanStep> LimitStep::copy(ContextPtr) const
{
return std::make_shared<LimitStep>(input_streams[0], limit, offset, always_read_till_end, with_ties, description, partial);
}
void LimitStep::prepare(const PreparedStatementContext & prepared_context)
{
prepared_context.prepare(limit);
prepared_context.prepare(offset);
}
}

View File

@ -14,8 +14,9 @@
*/
#pragma once
#include <QueryPlan/ITransformingStep.h>
#include <Core/SortDescription.h>
#include <Interpreters/prepared_statement.h>
#include <QueryPlan/ITransformingStep.h>
namespace DB
{
@ -26,8 +27,8 @@ class LimitStep : public ITransformingStep
public:
LimitStep(
const DataStream & input_stream_,
size_t limit_,
size_t offset_,
SizeOrVariable limit_,
SizeOrVariable offset_,
bool always_read_till_end_ = false, /// Read all data even if limit is reached. Needed for totals.
bool with_ties_ = false, /// Limit with ties.
SortDescription description_ = {},
@ -44,13 +45,31 @@ public:
size_t getLimitForSorting() const
{
if (limit > std::numeric_limits<UInt64>::max() - offset)
if (getLimitValue() > std::numeric_limits<UInt64>::max() - getOffsetValue())
return 0;
return limit + offset;
return getLimitValue() + getOffsetValue();
}
const SizeOrVariable & getLimit() const
{
return limit;
}
const SizeOrVariable & getOffset() const
{
return offset;
}
size_t getLimitValue() const
{
return std::get<size_t>(limit);
}
size_t getOffsetValue() const
{
return std::get<size_t>(offset);
}
bool hasPreparedParam() const
{
return std::holds_alternative<String>(limit) || std::holds_alternative<String>(offset);
}
size_t getLimit() const { return limit; }
size_t getOffset() const { return offset; }
bool isAlwaysReadTillEnd() const { return always_read_till_end; }
bool isWithTies() const { return with_ties; }
const SortDescription & getSortDescription() const { return description; }
@ -67,9 +86,11 @@ public:
std::shared_ptr<IQueryPlanStep> copy(ContextPtr ptr) const override;
void setInputStreams(const DataStreams & input_streams_) override;
void prepare(const PreparedStatementContext & prepared_context) override;
private:
size_t limit;
size_t offset;
SizeOrVariable limit;
SizeOrVariable offset;
bool always_read_till_end;
bool with_ties;

View File

@ -46,7 +46,8 @@ UInt128 PlanCacheManager::hash(const ASTPtr & query_ast, ContextMutablePtr & con
String settings_string;
WriteBufferFromString buffer(settings_string);
settings.write(buffer);
const static std::unordered_set<String> whitelist{"enable_plan_cache", "force_plan_cache"};
settings.write(buffer, SettingsWriteFormat::DEFAULT, whitelist);
UInt128 key;
SipHash hash;

View File

@ -40,6 +40,14 @@ PlanNodePtr PlanNodeBase::getNodeById(PlanNodeId node_id) const
return nullptr;
}
void PlanNodeBase::prepare(const PreparedStatementContext & prepared_context)
{
for (const auto & child : children)
child->prepare(prepared_context);
getStep()->prepare(prepared_context);
}
#define PLAN_NODE_DEF(TYPE) \
template class PlanNode<TYPE##Step>;
APPLY_STEP_TYPES(PLAN_NODE_DEF)

View File

@ -112,6 +112,7 @@ public:
NameToType getOutputNamesToTypes() const { return getCurrentDataStream().header.getNamesToTypes(); }
Names getOutputNames() const { return getCurrentDataStream().header.getNames(); }
PlanNodePtr getNodeById(PlanNodeId node_id) const;
void prepare(const PreparedStatementContext & prepared_context);
static PlanNodePtr createPlanNode(
[[maybe_unused]] PlanNodeId id_,

View File

@ -822,18 +822,37 @@ String PlanPrinter::TextPrinter::printDetail(QueryPlanStepPtr plan, const TextPr
for (const auto & desc : sort->getSortDescription())
sort_columns.emplace_back(desc.format());
out << intent.detailIntent() << "Order by: " << join(sort_columns, ", ", "{", "}");
if (sort->getLimit())
out << intent.detailIntent() << "Limit: " << sort->getLimit();
std::visit(
overloaded{
[&](size_t x) {
if (x)
out << intent.detailIntent() << "Limit: " << x;
},
[&](const String & x) { out << intent.detailIntent() << "Limit: " << x; }},
sort->getLimit());
}
if (verbose && plan->getType() == IQueryPlanStep::Type::Limit)
{
const auto * limit = dynamic_cast<const LimitStep *>(plan.get());
out << intent.detailIntent();
if (limit->getLimit())
out << "Limit: " << limit->getLimit();
if (limit->getOffset())
out << " Offset: " << limit->getOffset();
std::visit([&](const auto & v) { out << "Limit: " << v; }, limit->getLimit());
std::visit(
[&](const auto & v) {
using T = std::decay_t<decltype(v)>;
if constexpr (std::is_same_v<T, size_t>)
{
if (v)
out << " Offset: " << v;
}
else
{
out << " Offset: " << v;
}
},
limit->getOffset());
}
if (verbose && plan->getType() == IQueryPlanStep::Type::Offset)
@ -1094,18 +1113,39 @@ void NodeDescription::setStepDetail(QueryPlanStepPtr step)
const auto * sort = dynamic_cast<const SortingStep *>(step.get());
std::vector<String> sort_columns;
for (const auto & desc : sort->getSortDescription())
step_vector_detail["OrderBy"].emplace_back(desc.format());
if (sort->getLimit())
step_detail["Limit"] = std::to_string(sort->getLimit());
step_vector_detail["OrderBy"].emplace_back(
desc.column_name + (desc.direction == -1 ? " desc" : " asc") + (desc.nulls_direction == -1 ? " nulls_last" : ""));
std::visit(
overloaded{
[&](size_t x) {
if (x)
step_detail["Limit"] = std::to_string(x);
},
[&](const String & x) { step_detail["Limit"] = x; }},
sort->getLimit());
}
if (step->getType() == IQueryPlanStep::Type::Limit)
{
const auto * limit = dynamic_cast<const LimitStep *>(step.get());
if (limit->getLimit())
step_detail["Limit"] = std::to_string(limit->getLimit());
if (limit->getOffset())
step_detail["Offset"] = std::to_string(limit->getOffset());
std::visit(
[&](const auto & e) {
using T = std::decay_t<decltype(e)>;
if constexpr (std::is_same_v<T, size_t>)
step_detail["Limit"] = std::to_string(e);
else
step_detail["Limit"] = e;
},
limit->getLimit());
std::visit(
[&](const auto & e) {
using T = std::decay_t<decltype(e)>;
if constexpr (std::is_same_v<T, size_t>)
step_detail["Offset"] = std::to_string(e);
else
step_detail["Offset"] = e;
},
limit->getOffset());
}
if (step->getType() == IQueryPlanStep::Type::Offset)

View File

@ -83,7 +83,7 @@ std::shared_ptr<ProjectionStep> ProjectionStep::fromProto(const Protos::Projecti
std::shared_ptr<IQueryPlanStep> ProjectionStep::copy(ContextPtr) const
{
return std::make_shared<ProjectionStep>(input_streams[0], assignments, name_to_type, final_project, index_project, hints);
return std::make_shared<ProjectionStep>(input_streams[0], assignments.copy(), name_to_type, final_project, index_project, hints);
}
ActionsDAGPtr ProjectionStep::createActions(ContextPtr context) const
@ -112,4 +112,9 @@ ActionsDAGPtr ProjectionStep::createActions(const Assignments & assignments, con
return createExpressionActions(context, source, output, expr_list);
}
void ProjectionStep::prepare(const PreparedStatementContext & prepared_context)
{
for (auto & assign : assignments)
prepared_context.prepare(assign.second);
}
}

View File

@ -54,6 +54,8 @@ public:
std::shared_ptr<IQueryPlanStep> copy(ContextPtr) const override;
void setInputStreams(const DataStreams & input_streams_) override;
void prepare(const PreparedStatementContext & prepared_context) override;
static ActionsDAGPtr createActions(const Assignments & assignments, const NamesAndTypesList & source, ContextPtr context);
private:

View File

@ -1015,4 +1015,13 @@ QueryPlanPtr QueryPlan::copy(ContextMutablePtr context)
copy_cte_info.add(cte_id, copyPlanNode(cte_def, context));
return std::make_unique<QueryPlan>(copy_plan_node, copy_cte_info, context->getPlanNodeIdAllocator());
}
void QueryPlan::prepare(const PreparedStatementContext & prepared_context)
{
if (plan_node)
plan_node->prepare(prepared_context);
for (const auto & [cte_id, cte_def] : cte_info.getCTEs())
cte_def->prepare(prepared_context);
}
}

View File

@ -17,6 +17,7 @@
#include <Core/Names.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/prepared_statement.h>
#include <QueryPlan/CTEInfo.h>
#include <QueryPlan/PlanNodeIdAllocator.h>
#include <Interpreters/StorageID.h>
@ -192,6 +193,8 @@ public:
void setResetStepId(bool reset_id) { reset_step_id = reset_id; }
QueryPlanPtr copy(ContextMutablePtr context);
void prepare(const PreparedStatementContext & prepared_context);
private:
Poco::Logger * log = &Poco::Logger::get("QueryPlan");
// Flatten, in segment only

View File

@ -34,6 +34,7 @@
#include <Optimizer/SymbolsExtractor.h>
#include <Optimizer/Utils.h>
#include <Optimizer/makeCastFunction.h>
#include <Parsers/ASTPreparedStatement.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/formatAST.h>
#include <QueryPlan/AggregatingStep.h>
@ -103,6 +104,12 @@ public:
RelationPlan visitASTSelectQuery(ASTPtr & node, const Void &) override;
RelationPlan visitASTSubquery(ASTPtr & node, const Void &) override;
RelationPlan visitASTExplainQuery(ASTPtr & node, const Void &) override;
RelationPlan visitASTCreatePreparedStatementQuery(ASTPtr & node, const Void &) override
{
auto & prepare = node->as<ASTCreatePreparedStatementQuery &>();
auto query = prepare.getQuery();
return process(query);
}
RelationPlan process(ASTPtr & node) { return ASTVisitorUtil::accept(node, *this, {}); }

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