mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #42284 from Algunenano/perf_experiment
Performance experiment
This commit is contained in:
commit
dfe93b5d82
@ -30,9 +30,10 @@
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/UseSSL.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
@ -41,6 +42,8 @@
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
|
||||
#include <Processors/Transforms/getSourceFromASTInsertQuery.h>
|
||||
|
||||
#include <Interpreters/InterpreterSetQuery.h>
|
||||
|
||||
#include <Functions/registerFunctions.h>
|
||||
@ -827,6 +830,20 @@ bool Client::processWithFuzzing(const String & full_query)
|
||||
WriteBufferFromOStream ast_buf(std::cout, 4096);
|
||||
formatAST(*query, ast_buf, false /*highlight*/);
|
||||
ast_buf.next();
|
||||
if (const auto * insert = query->as<ASTInsertQuery>())
|
||||
{
|
||||
/// For inserts with data it's really useful to have the data itself available in the logs, as formatAST doesn't print it
|
||||
if (insert->hasInlinedData())
|
||||
{
|
||||
String bytes;
|
||||
{
|
||||
auto read_buf = getReadBufferFromASTInsertQuery(query);
|
||||
WriteBufferFromString write_buf(bytes);
|
||||
copyData(*read_buf, write_buf);
|
||||
}
|
||||
std::cout << std::endl << bytes;
|
||||
}
|
||||
}
|
||||
std::cout << std::endl << std::endl;
|
||||
|
||||
try
|
||||
|
@ -396,6 +396,7 @@ if (TARGET ch_contrib::cpuid)
|
||||
endif()
|
||||
|
||||
dbms_target_link_libraries(PUBLIC ch_contrib::abseil_swiss_tables)
|
||||
target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::abseil_swiss_tables)
|
||||
|
||||
# Make dbms depend on roaring instead of clickhouse_common_io so that roaring itself can depend on clickhouse_common_io
|
||||
# That way we we can redirect malloc/free functions avoiding circular dependencies
|
||||
|
@ -327,9 +327,7 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast)
|
||||
// Add element
|
||||
if (fuzz_rand() % 50 == 0)
|
||||
{
|
||||
auto pos = list->children.empty()
|
||||
? list->children.begin()
|
||||
: list->children.begin() + fuzz_rand() % list->children.size();
|
||||
auto * pos = list->children.empty() ? list->children.begin() : list->children.begin() + fuzz_rand() % list->children.size();
|
||||
auto col = getRandomColumnLike();
|
||||
if (col)
|
||||
{
|
||||
@ -373,9 +371,7 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast)
|
||||
// Add element
|
||||
if (fuzz_rand() % 50 == 0)
|
||||
{
|
||||
auto pos = impl->children.empty()
|
||||
? impl->children.begin()
|
||||
: impl->children.begin() + fuzz_rand() % impl->children.size();
|
||||
auto * pos = impl->children.empty() ? impl->children.begin() : impl->children.begin() + fuzz_rand() % impl->children.size();
|
||||
auto col = getRandomColumnLike();
|
||||
if (col)
|
||||
impl->children.insert(pos, col);
|
||||
|
@ -188,13 +188,10 @@ void ThreadStatus::updatePerformanceCounters()
|
||||
}
|
||||
}
|
||||
|
||||
void ThreadStatus::assertState(const std::initializer_list<int> & permitted_states, const char * description) const
|
||||
void ThreadStatus::assertState(ThreadState permitted_state, const char * description) const
|
||||
{
|
||||
for (auto permitted_state : permitted_states)
|
||||
{
|
||||
if (getCurrentState() == permitted_state)
|
||||
return;
|
||||
}
|
||||
if (getCurrentState() == permitted_state)
|
||||
return;
|
||||
|
||||
if (description)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", getCurrentState(), description);
|
||||
|
@ -87,10 +87,6 @@ public:
|
||||
LogsLevel client_logs_level = LogsLevel::none;
|
||||
|
||||
String query;
|
||||
/// Query without new lines (see toOneLineQuery())
|
||||
/// Used to print in case of fatal error
|
||||
/// (to avoid calling extra code in the fatal error handler)
|
||||
String one_line_query;
|
||||
UInt64 normalized_query_hash = 0;
|
||||
|
||||
std::vector<ProfileEventsCountersAndMemory> finished_threads_counters_memory;
|
||||
@ -296,7 +292,7 @@ protected:
|
||||
void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point<std::chrono::system_clock> now);
|
||||
|
||||
|
||||
void assertState(const std::initializer_list<int> & permitted_states, const char * description = nullptr) const;
|
||||
void assertState(ThreadState permitted_state, const char * description = nullptr) const;
|
||||
|
||||
|
||||
private:
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
#include <Daemon/BaseDaemon.h>
|
||||
#include <Daemon/SentryWriter.h>
|
||||
#include <Parsers/toOneLineQuery.h>
|
||||
#include <base/errnoToString.h>
|
||||
|
||||
#include <sys/stat.h>
|
||||
@ -303,7 +304,7 @@ private:
|
||||
|
||||
if (auto thread_group = thread_ptr->getThreadGroup())
|
||||
{
|
||||
query = thread_group->one_line_query;
|
||||
query = DB::toOneLineQuery(thread_group->query);
|
||||
}
|
||||
|
||||
if (auto logs_queue = thread_ptr->getInternalTextLogsQueue())
|
||||
|
@ -11,7 +11,7 @@ if (OS_DARWIN AND NOT USE_STATIC_LIBRARIES)
|
||||
target_link_libraries (daemon PUBLIC -Wl,-undefined,dynamic_lookup)
|
||||
endif()
|
||||
|
||||
target_link_libraries (daemon PUBLIC loggers common PRIVATE clickhouse_common_io clickhouse_common_config)
|
||||
target_link_libraries (daemon PUBLIC loggers common PRIVATE clickhouse_parsers clickhouse_common_io clickhouse_common_config)
|
||||
|
||||
if (TARGET ch_contrib::sentry)
|
||||
target_link_libraries (daemon PRIVATE ch_contrib::sentry dbms)
|
||||
|
@ -344,11 +344,14 @@ void buildPrimaryKeyConfiguration(
|
||||
|
||||
auto identifier_name = key_names.front();
|
||||
|
||||
auto it = std::find_if(children.begin(), children.end(), [&](const ASTPtr & node)
|
||||
{
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr = node->as<const ASTDictionaryAttributeDeclaration>();
|
||||
return dict_attr->name == identifier_name;
|
||||
});
|
||||
const auto * it = std::find_if(
|
||||
children.begin(),
|
||||
children.end(),
|
||||
[&](const ASTPtr & node)
|
||||
{
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr = node->as<const ASTDictionaryAttributeDeclaration>();
|
||||
return dict_attr->name == identifier_name;
|
||||
});
|
||||
|
||||
if (it == children.end())
|
||||
{
|
||||
|
@ -88,7 +88,7 @@ void ApplyWithGlobalVisitor::visit(ASTPtr & ast)
|
||||
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(child.get()))
|
||||
exprs[ast_with_alias->alias] = child;
|
||||
}
|
||||
for (auto it = node_union->list_of_selects->children.begin() + 1; it != node_union->list_of_selects->children.end(); ++it)
|
||||
for (auto * it = node_union->list_of_selects->children.begin() + 1; it != node_union->list_of_selects->children.end(); ++it)
|
||||
{
|
||||
if (auto * union_child = (*it)->as<ASTSelectWithUnionQuery>())
|
||||
visit(*union_child, exprs, with_expression_list);
|
||||
|
@ -17,7 +17,7 @@ class ComparisonGraph
|
||||
{
|
||||
public:
|
||||
/// atomic_formulas are extracted from constraints.
|
||||
explicit ComparisonGraph(const std::vector<ASTPtr> & atomic_formulas);
|
||||
explicit ComparisonGraph(const ASTs & atomic_formulas);
|
||||
|
||||
enum class CompareResult
|
||||
{
|
||||
@ -43,7 +43,7 @@ public:
|
||||
bool isAlwaysCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const;
|
||||
|
||||
/// Returns all expressions from component to which @ast belongs if any.
|
||||
std::vector<ASTPtr> getEqual(const ASTPtr & ast) const;
|
||||
ASTs getEqual(const ASTPtr & ast) const;
|
||||
|
||||
/// Returns constant expression from component to which @ast belongs if any.
|
||||
std::optional<ASTPtr> getEqualConst(const ASTPtr & ast) const;
|
||||
@ -52,7 +52,7 @@ public:
|
||||
std::optional<std::size_t> getComponentId(const ASTPtr & ast) const;
|
||||
|
||||
/// Returns all expressions from component.
|
||||
std::vector<ASTPtr> getComponent(size_t id) const;
|
||||
ASTs getComponent(size_t id) const;
|
||||
|
||||
size_t getNumOfComponents() const { return graph.vertices.size(); }
|
||||
|
||||
@ -72,7 +72,7 @@ private:
|
||||
struct EqualComponent
|
||||
{
|
||||
/// All these expressions are considered as equal.
|
||||
std::vector<ASTPtr> asts;
|
||||
ASTs asts;
|
||||
std::optional<size_t> constant_index;
|
||||
|
||||
bool hasConstant() const;
|
||||
|
@ -149,7 +149,7 @@ ASTPtr makeOnExpression(const std::vector<ASTPtr> & expressions)
|
||||
if (expressions.size() == 1)
|
||||
return expressions[0]->clone();
|
||||
|
||||
std::vector<ASTPtr> arguments;
|
||||
ASTs arguments;
|
||||
arguments.reserve(expressions.size());
|
||||
for (const auto & ast : expressions)
|
||||
arguments.emplace_back(ast->clone());
|
||||
|
@ -348,7 +348,7 @@ void splitConjunctionsAst(const ASTPtr & node, ASTs & result)
|
||||
|
||||
ASTs splitConjunctionsAst(const ASTPtr & node)
|
||||
{
|
||||
std::vector<ASTPtr> result;
|
||||
ASTs result;
|
||||
splitConjunctionsAst(node, result);
|
||||
return result;
|
||||
}
|
||||
|
@ -559,7 +559,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
|
||||
ColumnsDescription res;
|
||||
auto name_type_it = column_names_and_types.begin();
|
||||
for (auto ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it)
|
||||
for (const auto * ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it)
|
||||
{
|
||||
ColumnDescription column;
|
||||
|
||||
|
@ -153,7 +153,7 @@ private:
|
||||
data.addTableColumns(identifier.name(), columns);
|
||||
|
||||
// QualifiedAsterisk's transformers start to appear at child 1
|
||||
for (auto it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
|
||||
for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
|
||||
{
|
||||
IASTColumnsTransformer::transform(*it, columns);
|
||||
}
|
||||
@ -209,7 +209,7 @@ struct RewriteTablesVisitorData
|
||||
{
|
||||
if (done)
|
||||
return;
|
||||
std::vector<ASTPtr> new_tables{left, right};
|
||||
ASTs new_tables{left, right};
|
||||
ast->children.swap(new_tables);
|
||||
done = true;
|
||||
}
|
||||
|
@ -313,7 +313,7 @@ void LogicalExpressionsOptimizer::cleanupOrExpressions()
|
||||
for (const auto & entry : garbage_map)
|
||||
{
|
||||
const auto * function = entry.first;
|
||||
auto first_erased = entry.second;
|
||||
auto * first_erased = entry.second;
|
||||
|
||||
auto & operands = getFunctionOperands(function);
|
||||
operands.erase(first_erased, operands.end());
|
||||
|
@ -155,7 +155,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col
|
||||
/// but this produce endless recursion in gcc-11, and leads to SIGSEGV
|
||||
/// (see git blame for details).
|
||||
auto column_name_and_type = columns_name_and_type.begin();
|
||||
auto declare_column_ast = columns_definition->children.begin();
|
||||
const auto * declare_column_ast = columns_definition->children.begin();
|
||||
for (; column_name_and_type != columns_name_and_type.end(); column_name_and_type++, declare_column_ast++)
|
||||
{
|
||||
const auto & declare_column = (*declare_column_ast)->as<MySQLParser::ASTDeclareColumn>();
|
||||
|
@ -219,7 +219,6 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
|
||||
user_process_list.user_temp_data_on_disk, settings.max_temporary_data_on_disk_size_for_query));
|
||||
}
|
||||
thread_group->query = query_;
|
||||
thread_group->one_line_query = toOneLineQuery(query_);
|
||||
thread_group->normalized_query_hash = normalizedQueryHash<false>(query_);
|
||||
|
||||
/// Set query-level memory trackers
|
||||
|
@ -97,7 +97,7 @@ void CurrentThread::defaultThreadDeleter()
|
||||
|
||||
void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_)
|
||||
{
|
||||
assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__);
|
||||
assertState(ThreadState::DetachedFromQuery, __PRETTY_FUNCTION__);
|
||||
|
||||
/// Attach or init current thread to thread group and copy useful information from it
|
||||
thread_group = thread_group_;
|
||||
@ -324,7 +324,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
|
||||
return;
|
||||
}
|
||||
|
||||
assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__);
|
||||
assertState(ThreadState::AttachedToQuery, __PRETTY_FUNCTION__);
|
||||
|
||||
finalizeQueryProfiler();
|
||||
finalizePerformanceCounters();
|
||||
|
@ -299,7 +299,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt
|
||||
}
|
||||
|
||||
// QualifiedAsterisk's transformers start to appear at child 1
|
||||
for (auto it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
|
||||
for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it)
|
||||
{
|
||||
IASTColumnsTransformer::transform(*it, columns);
|
||||
}
|
||||
|
@ -573,7 +573,7 @@ void removeUnneededColumnsFromSelectClause(ASTSelectQuery * select_query, const
|
||||
auto & children = select_query->interpolate()->children;
|
||||
if (!children.empty())
|
||||
{
|
||||
for (auto it = children.begin(); it != children.end();)
|
||||
for (auto * it = children.begin(); it != children.end();)
|
||||
{
|
||||
if (remove_columns.contains((*it)->as<ASTInterpolateElement>()->column))
|
||||
it = select_query->interpolate()->children.erase(it);
|
||||
|
@ -26,10 +26,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat
|
||||
if (!create_query->columns_list->columns)
|
||||
create_query->columns_list->set(create_query->columns_list->columns, std::make_shared<ASTExpressionList>());
|
||||
auto & dest_children = create_query->columns_list->columns->children;
|
||||
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
|
||||
{
|
||||
return node->as<ASTColumnDeclaration>()->name == override_column->name;
|
||||
});
|
||||
auto * exists = std::find_if(
|
||||
dest_children.begin(),
|
||||
dest_children.end(),
|
||||
[&](ASTPtr node) -> bool { return node->as<ASTColumnDeclaration>()->name == override_column->name; });
|
||||
/// For columns, only allow adding ALIAS (non-physical) for now.
|
||||
/// TODO: This logic should instead be handled by validation that is
|
||||
/// executed from InterpreterCreateQuery / InterpreterAlterQuery.
|
||||
@ -52,10 +52,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat
|
||||
if (!create_query->columns_list->indices)
|
||||
create_query->columns_list->set(create_query->columns_list->indices, std::make_shared<ASTExpressionList>());
|
||||
auto & dest_children = create_query->columns_list->indices->children;
|
||||
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
|
||||
{
|
||||
return node->as<ASTIndexDeclaration>()->name == override_index->name;
|
||||
});
|
||||
auto * exists = std::find_if(
|
||||
dest_children.begin(),
|
||||
dest_children.end(),
|
||||
[&](ASTPtr node) -> bool { return node->as<ASTIndexDeclaration>()->name == override_index->name; });
|
||||
if (exists == dest_children.end())
|
||||
dest_children.emplace_back(override_index_ast);
|
||||
else
|
||||
@ -72,10 +72,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat
|
||||
if (!create_query->columns_list->constraints)
|
||||
create_query->columns_list->set(create_query->columns_list->constraints, std::make_shared<ASTExpressionList>());
|
||||
auto & dest_children = create_query->columns_list->constraints->children;
|
||||
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
|
||||
{
|
||||
return node->as<ASTConstraintDeclaration>()->name == override_constraint->name;
|
||||
});
|
||||
auto * exists = std::find_if(
|
||||
dest_children.begin(),
|
||||
dest_children.end(),
|
||||
[&](ASTPtr node) -> bool { return node->as<ASTConstraintDeclaration>()->name == override_constraint->name; });
|
||||
if (exists == dest_children.end())
|
||||
dest_children.emplace_back(override_constraint_ast);
|
||||
else
|
||||
@ -92,10 +92,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat
|
||||
if (!create_query->columns_list->projections)
|
||||
create_query->columns_list->set(create_query->columns_list->projections, std::make_shared<ASTExpressionList>());
|
||||
auto & dest_children = create_query->columns_list->projections->children;
|
||||
auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool
|
||||
{
|
||||
return node->as<ASTProjectionDeclaration>()->name == override_projection->name;
|
||||
});
|
||||
auto * exists = std::find_if(
|
||||
dest_children.begin(),
|
||||
dest_children.end(),
|
||||
[&](ASTPtr node) -> bool { return node->as<ASTProjectionDeclaration>()->name == override_projection->name; });
|
||||
if (exists == dest_children.end())
|
||||
dest_children.emplace_back(override_projection_ast);
|
||||
else
|
||||
|
@ -87,7 +87,7 @@ void ASTColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) const
|
||||
void ASTColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
{
|
||||
writeCString("COLUMNS(", ostr);
|
||||
for (auto it = column_list->children.begin(); it != column_list->children.end(); ++it)
|
||||
for (auto * it = column_list->children.begin(); it != column_list->children.end(); ++it)
|
||||
{
|
||||
if (it != column_list->children.begin())
|
||||
writeCString(", ", ostr);
|
||||
@ -198,7 +198,7 @@ void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const
|
||||
qualifier->appendColumnName(ostr);
|
||||
writeCString(".COLUMNS(", ostr);
|
||||
|
||||
for (auto it = column_list->children.begin(); it != column_list->children.end(); ++it)
|
||||
for (auto * it = column_list->children.begin(); it != column_list->children.end(); ++it)
|
||||
{
|
||||
if (it != column_list->children.begin())
|
||||
writeCString(", ", ostr);
|
||||
|
@ -217,7 +217,7 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const
|
||||
for (const auto & child : children)
|
||||
expected_columns.insert(child->as<const ASTIdentifier &>().name());
|
||||
|
||||
for (auto it = nodes.begin(); it != nodes.end();)
|
||||
for (auto * it = nodes.begin(); it != nodes.end();)
|
||||
{
|
||||
if (const auto * id = it->get()->as<ASTIdentifier>())
|
||||
{
|
||||
@ -234,7 +234,7 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = nodes.begin(); it != nodes.end();)
|
||||
for (auto * it = nodes.begin(); it != nodes.end();)
|
||||
{
|
||||
if (const auto * id = it->get()->as<ASTIdentifier>())
|
||||
{
|
||||
|
@ -383,7 +383,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
||||
if (parameters)
|
||||
{
|
||||
writeChar('(', ostr);
|
||||
for (auto it = parameters->children.begin(); it != parameters->children.end(); ++it)
|
||||
for (auto * it = parameters->children.begin(); it != parameters->children.end(); ++it)
|
||||
{
|
||||
if (it != parameters->children.begin())
|
||||
writeCString(", ", ostr);
|
||||
@ -396,7 +396,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const
|
||||
writeChar('(', ostr);
|
||||
if (arguments)
|
||||
{
|
||||
for (auto it = arguments->children.begin(); it != arguments->children.end(); ++it)
|
||||
for (auto * it = arguments->children.begin(); it != arguments->children.end(); ++it)
|
||||
{
|
||||
if (it != arguments->children.begin())
|
||||
writeCString(", ", ostr);
|
||||
|
@ -24,7 +24,7 @@ ASTIdentifier::ASTIdentifier(const String & short_name, ASTPtr && name_param)
|
||||
children.push_back(std::move(name_param));
|
||||
}
|
||||
|
||||
ASTIdentifier::ASTIdentifier(std::vector<String> && name_parts_, bool special, std::vector<ASTPtr> && name_params)
|
||||
ASTIdentifier::ASTIdentifier(std::vector<String> && name_parts_, bool special, ASTs && name_params)
|
||||
: name_parts(name_parts_), semantic(std::make_shared<IdentifierSemanticImpl>())
|
||||
{
|
||||
assert(!name_parts.empty());
|
||||
@ -164,12 +164,12 @@ void ASTIdentifier::resetFullName()
|
||||
full_name += '.' + name_parts[i];
|
||||
}
|
||||
|
||||
ASTTableIdentifier::ASTTableIdentifier(const String & table_name, std::vector<ASTPtr> && name_params)
|
||||
ASTTableIdentifier::ASTTableIdentifier(const String & table_name, ASTs && name_params)
|
||||
: ASTIdentifier({table_name}, true, std::move(name_params))
|
||||
{
|
||||
}
|
||||
|
||||
ASTTableIdentifier::ASTTableIdentifier(const StorageID & table_id, std::vector<ASTPtr> && name_params)
|
||||
ASTTableIdentifier::ASTTableIdentifier(const StorageID & table_id, ASTs && name_params)
|
||||
: ASTIdentifier(
|
||||
table_id.database_name.empty() ? std::vector<String>{table_id.table_name}
|
||||
: std::vector<String>{table_id.database_name, table_id.table_name},
|
||||
@ -178,7 +178,7 @@ ASTTableIdentifier::ASTTableIdentifier(const StorageID & table_id, std::vector<A
|
||||
uuid = table_id.uuid;
|
||||
}
|
||||
|
||||
ASTTableIdentifier::ASTTableIdentifier(const String & database_name, const String & table_name, std::vector<ASTPtr> && name_params)
|
||||
ASTTableIdentifier::ASTTableIdentifier(const String & database_name, const String & table_name, ASTs && name_params)
|
||||
: ASTIdentifier({database_name, table_name}, true, std::move(name_params))
|
||||
{
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/ASTIdentifier_fwd.h>
|
||||
#include <Parsers/ASTQueryParameter.h>
|
||||
#include <Parsers/ASTWithAlias.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -23,7 +24,7 @@ class ASTIdentifier : public ASTWithAlias
|
||||
{
|
||||
public:
|
||||
explicit ASTIdentifier(const String & short_name, ASTPtr && name_param = {});
|
||||
explicit ASTIdentifier(std::vector<String> && name_parts, bool special = false, std::vector<ASTPtr> && name_params = {});
|
||||
explicit ASTIdentifier(std::vector<String> && name_parts, bool special = false, ASTs && name_params = {});
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char delim) const override { return "Identifier" + (delim + name()); }
|
||||
@ -72,9 +73,9 @@ private:
|
||||
class ASTTableIdentifier : public ASTIdentifier
|
||||
{
|
||||
public:
|
||||
explicit ASTTableIdentifier(const String & table_name, std::vector<ASTPtr> && name_params = {});
|
||||
explicit ASTTableIdentifier(const StorageID & table_id, std::vector<ASTPtr> && name_params = {});
|
||||
ASTTableIdentifier(const String & database_name, const String & table_name, std::vector<ASTPtr> && name_params = {});
|
||||
explicit ASTTableIdentifier(const String & table_name, ASTs && name_params = {});
|
||||
explicit ASTTableIdentifier(const StorageID & table_id, ASTs && name_params = {});
|
||||
ASTTableIdentifier(const String & database_name, const String & table_name, ASTs && name_params = {});
|
||||
|
||||
String getID(char delim) const override { return "TableIdentifier" + (delim + name()); }
|
||||
ASTPtr clone() const override;
|
||||
|
@ -52,7 +52,7 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st
|
||||
else if (mode == TTLMode::GROUP_BY)
|
||||
{
|
||||
settings.ostr << " GROUP BY ";
|
||||
for (auto it = group_by_key.begin(); it != group_by_key.end(); ++it)
|
||||
for (const auto * it = group_by_key.begin(); it != group_by_key.end(); ++it)
|
||||
{
|
||||
if (it != group_by_key.begin())
|
||||
settings.ostr << ", ";
|
||||
@ -62,7 +62,7 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st
|
||||
if (!group_by_assignments.empty())
|
||||
{
|
||||
settings.ostr << " SET ";
|
||||
for (auto it = group_by_assignments.begin(); it != group_by_assignments.end(); ++it)
|
||||
for (const auto * it = group_by_assignments.begin(); it != group_by_assignments.end(); ++it)
|
||||
{
|
||||
if (it != group_by_assignments.begin())
|
||||
settings.ostr << ", ";
|
||||
|
@ -37,6 +37,7 @@
|
||||
#include <Parsers/parseIdentifierOrStringLiteral.h>
|
||||
#include <Parsers/parseIntervalKind.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Parsers/ParserSelectWithUnionQuery.h>
|
||||
#include <Parsers/ParserCase.h>
|
||||
|
||||
@ -224,7 +225,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
|
||||
return false;
|
||||
|
||||
std::vector<String> parts;
|
||||
std::vector<ASTPtr> params;
|
||||
ASTs params;
|
||||
const auto & list = id_list->as<ASTExpressionList &>();
|
||||
for (const auto & child : list.children)
|
||||
{
|
||||
|
@ -46,16 +46,15 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!elem_parser->parse(pos, element, expected))
|
||||
return false;
|
||||
|
||||
elements.push_back(element);
|
||||
elements.push_back(std::move(element));
|
||||
return true;
|
||||
};
|
||||
|
||||
if (!parseUtil(pos, expected, parse_element, *separator_parser, allow_empty))
|
||||
return false;
|
||||
|
||||
auto list = std::make_shared<ASTExpressionList>(result_separator);
|
||||
list->children = std::move(elements);
|
||||
node = list;
|
||||
node = std::make_shared<ASTExpressionList>(result_separator);
|
||||
node->children = std::move(elements);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -76,7 +75,7 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!elem_parser.parse(pos, element, expected))
|
||||
return false;
|
||||
|
||||
elements.push_back(element);
|
||||
elements.push_back(std::move(element));
|
||||
return true;
|
||||
};
|
||||
|
||||
@ -120,9 +119,8 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!parseUtil(pos, parse_element, parse_separator))
|
||||
return false;
|
||||
|
||||
auto list = std::make_shared<ASTExpressionList>();
|
||||
list->children = std::move(elements);
|
||||
node = list;
|
||||
node = std::make_shared<ASTExpressionList>();
|
||||
node->children = std::move(elements);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -242,7 +240,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node
|
||||
if (!elem_parser->parse(pos, elem, expected))
|
||||
return false;
|
||||
|
||||
node = elem;
|
||||
node = std::move(elem);
|
||||
first = false;
|
||||
}
|
||||
else
|
||||
@ -607,7 +605,7 @@ public:
|
||||
|
||||
asts.reserve(asts.size() + n);
|
||||
|
||||
auto start = operands.begin() + operands.size() - n;
|
||||
auto * start = operands.begin() + operands.size() - n;
|
||||
asts.insert(asts.end(), std::make_move_iterator(start), std::make_move_iterator(operands.end()));
|
||||
operands.erase(start, operands.end());
|
||||
|
||||
@ -1459,7 +1457,7 @@ public:
|
||||
return false;
|
||||
|
||||
auto subquery = std::make_shared<ASTSubquery>();
|
||||
subquery->children.push_back(node);
|
||||
subquery->children.push_back(std::move(node));
|
||||
elements = {makeASTFunction("exists", subquery)};
|
||||
|
||||
finished = true;
|
||||
|
@ -1,13 +1,36 @@
|
||||
#pragma once
|
||||
|
||||
#include <algorithm>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <absl/container/inlined_vector.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IAST;
|
||||
using ASTPtr = std::shared_ptr<IAST>;
|
||||
using ASTs = std::vector<ASTPtr>;
|
||||
/// sizeof(absl::InlinedVector<ASTPtr, N>) == 8 + N * 16.
|
||||
/// 7 elements take 120 Bytes which is ~128
|
||||
using ASTs = absl::InlinedVector<ASTPtr, 7>;
|
||||
|
||||
}
|
||||
|
||||
namespace std
|
||||
{
|
||||
|
||||
inline typename DB::ASTs::size_type erase(DB::ASTs & asts, const DB::ASTPtr & element)
|
||||
{
|
||||
auto old_size = asts.size();
|
||||
asts.erase(std::remove(asts.begin(), asts.end(), element), asts.end());
|
||||
return old_size - asts.size();
|
||||
}
|
||||
|
||||
template <class Predicate>
|
||||
inline typename DB::ASTs::size_type erase_if(DB::ASTs & asts, Predicate pred)
|
||||
{
|
||||
auto old_size = asts.size();
|
||||
asts.erase(std::remove_if(asts.begin(), asts.end(), pred), asts.end());
|
||||
return old_size - asts.size();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <absl/container/inlined_vector.h>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
@ -24,8 +24,8 @@ namespace ErrorCodes
|
||||
*/
|
||||
struct Expected
|
||||
{
|
||||
absl::InlinedVector<const char *, 7> variants;
|
||||
const char * max_parsed_pos = nullptr;
|
||||
std::vector<const char *> variants;
|
||||
|
||||
/// 'description' should be statically allocated string.
|
||||
ALWAYS_INLINE void add(const char * current_pos, const char * description)
|
||||
@ -38,7 +38,7 @@ struct Expected
|
||||
return;
|
||||
}
|
||||
|
||||
if ((current_pos == max_parsed_pos) && (find(variants.begin(), variants.end(), description) == variants.end()))
|
||||
if ((current_pos == max_parsed_pos) && (std::find(variants.begin(), variants.end(), description) == variants.end()))
|
||||
variants.push_back(description);
|
||||
}
|
||||
|
||||
@ -64,6 +64,8 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
Pos(TokenIterator token_iterator_, uint32_t max_depth_) : TokenIterator(token_iterator_), max_depth(max_depth_) { }
|
||||
|
||||
ALWAYS_INLINE void increaseDepth()
|
||||
{
|
||||
++depth;
|
||||
|
@ -4,6 +4,20 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Tokens::Tokens(const char * begin, const char * end, size_t max_query_size)
|
||||
{
|
||||
Lexer lexer(begin, end, max_query_size);
|
||||
|
||||
bool stop = false;
|
||||
do
|
||||
{
|
||||
Token token = lexer.nextToken();
|
||||
stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded;
|
||||
if (token.isSignificant())
|
||||
data.emplace_back(std::move(token));
|
||||
} while (!stop);
|
||||
}
|
||||
|
||||
UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin)
|
||||
{
|
||||
/// We have just two kind of parentheses: () and [].
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Defines.h>
|
||||
#include <Parsers/Lexer.h>
|
||||
|
||||
#include <cassert>
|
||||
#include <vector>
|
||||
|
||||
|
||||
@ -20,34 +21,19 @@ class Tokens
|
||||
{
|
||||
private:
|
||||
std::vector<Token> data;
|
||||
Lexer lexer;
|
||||
std::size_t last_accessed_index = 0;
|
||||
|
||||
public:
|
||||
Tokens(const char * begin, const char * end, size_t max_query_size = 0) : lexer(begin, end, max_query_size) {}
|
||||
Tokens(const char * begin, const char * end, size_t max_query_size = 0);
|
||||
|
||||
const Token & operator[] (size_t index)
|
||||
ALWAYS_INLINE inline const Token & operator[](size_t index)
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
if (index < data.size())
|
||||
return data[index];
|
||||
|
||||
if (!data.empty() && data.back().isEnd())
|
||||
return data.back();
|
||||
|
||||
Token token = lexer.nextToken();
|
||||
|
||||
if (token.isSignificant())
|
||||
data.emplace_back(token);
|
||||
}
|
||||
assert(index < data.size());
|
||||
last_accessed_index = std::max(last_accessed_index, index);
|
||||
return data[index];
|
||||
}
|
||||
|
||||
const Token & max()
|
||||
{
|
||||
if (data.empty())
|
||||
return (*this)[0];
|
||||
return data.back();
|
||||
}
|
||||
ALWAYS_INLINE inline const Token & max() { return data[last_accessed_index]; }
|
||||
};
|
||||
|
||||
|
||||
|
@ -418,12 +418,13 @@ ConstantExpressionTemplate::Cache::getFromCacheOrConstruct(const DataTypePtr & r
|
||||
return res;
|
||||
}
|
||||
|
||||
bool ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const FormatSettings & format_settings, const Settings & settings)
|
||||
bool ConstantExpressionTemplate::parseExpression(
|
||||
ReadBuffer & istr, const TokenIterator & token_iterator, const FormatSettings & format_settings, const Settings & settings)
|
||||
{
|
||||
size_t cur_column = 0;
|
||||
try
|
||||
{
|
||||
if (tryParseExpression(istr, format_settings, cur_column, settings))
|
||||
if (tryParseExpression(istr, token_iterator, format_settings, cur_column, settings))
|
||||
{
|
||||
++rows_count;
|
||||
return true;
|
||||
@ -445,7 +446,12 @@ bool ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const Format
|
||||
return false;
|
||||
}
|
||||
|
||||
bool ConstantExpressionTemplate::tryParseExpression(ReadBuffer & istr, const FormatSettings & format_settings, size_t & cur_column, const Settings & settings)
|
||||
bool ConstantExpressionTemplate::tryParseExpression(
|
||||
ReadBuffer & istr,
|
||||
const TokenIterator & token_iterator,
|
||||
const FormatSettings & format_settings,
|
||||
size_t & cur_column,
|
||||
const Settings & settings)
|
||||
{
|
||||
size_t cur_token = 0;
|
||||
size_t num_columns = structure->literals.columns();
|
||||
@ -464,7 +470,7 @@ bool ConstantExpressionTemplate::tryParseExpression(ReadBuffer & istr, const For
|
||||
const DataTypePtr & type = structure->literals.getByPosition(cur_column).type;
|
||||
if (format_settings.values.accurate_types_of_literals && !structure->special_parser[cur_column].useDefaultParser())
|
||||
{
|
||||
if (!parseLiteralAndAssertType(istr, type.get(), cur_column, settings))
|
||||
if (!parseLiteralAndAssertType(istr, token_iterator, type.get(), cur_column, settings))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
@ -482,7 +488,8 @@ bool ConstantExpressionTemplate::tryParseExpression(ReadBuffer & istr, const For
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, const IDataType * complex_type, size_t column_idx, const Settings & settings)
|
||||
bool ConstantExpressionTemplate::parseLiteralAndAssertType(
|
||||
ReadBuffer & istr, const TokenIterator & token_iterator, const IDataType * complex_type, size_t column_idx, const Settings & settings)
|
||||
{
|
||||
using Type = Field::Types::Which;
|
||||
|
||||
@ -497,12 +504,12 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co
|
||||
|
||||
if (type_info.is_array || type_info.is_tuple || type_info.is_map)
|
||||
{
|
||||
/// TODO faster way to check types without using Parsers
|
||||
ParserArrayOfLiterals parser_array;
|
||||
ParserTupleOfLiterals parser_tuple;
|
||||
|
||||
Tokens tokens_number(istr.position(), istr.buffer().end());
|
||||
IParser::Pos iterator(tokens_number, static_cast<unsigned>(settings.max_parser_depth));
|
||||
IParser::Pos iterator(token_iterator, static_cast<unsigned>(settings.max_parser_depth));
|
||||
while (iterator->begin < istr.position())
|
||||
++iterator;
|
||||
Expected expected;
|
||||
ASTPtr ast;
|
||||
if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected))
|
||||
|
@ -71,7 +71,8 @@ public:
|
||||
|
||||
/// Read expression from istr, assert it has the same structure and the same types of literals (template matches)
|
||||
/// and parse literals into temporary columns
|
||||
bool parseExpression(ReadBuffer & istr, const FormatSettings & format_settings, const Settings & settings);
|
||||
bool parseExpression(
|
||||
ReadBuffer & istr, const TokenIterator & token_iterator, const FormatSettings & format_settings, const Settings & settings);
|
||||
|
||||
/// Evaluate batch of expressions were parsed using template.
|
||||
/// If template was deduced with null_as_default == true, set bits in nulls for NULL values in column_idx, starting from offset.
|
||||
@ -80,8 +81,14 @@ public:
|
||||
size_t rowsCount() const { return rows_count; }
|
||||
|
||||
private:
|
||||
bool tryParseExpression(ReadBuffer & istr, const FormatSettings & format_settings, size_t & cur_column, const Settings & settings);
|
||||
bool parseLiteralAndAssertType(ReadBuffer & istr, const IDataType * type, size_t column_idx, const Settings & settings);
|
||||
bool tryParseExpression(
|
||||
ReadBuffer & istr,
|
||||
const TokenIterator & token_iterator,
|
||||
const FormatSettings & format_settings,
|
||||
size_t & cur_column,
|
||||
const Settings & settings);
|
||||
bool parseLiteralAndAssertType(
|
||||
ReadBuffer & istr, const TokenIterator & token_iterator, const IDataType * type, size_t column_idx, const Settings & settings);
|
||||
|
||||
private:
|
||||
TemplateStructurePtr structure;
|
||||
|
@ -108,15 +108,97 @@ Chunk ValuesBlockInputFormat::generate()
|
||||
return Chunk{std::move(columns), rows_in_block};
|
||||
}
|
||||
|
||||
/// Can be used in fileSegmentationEngine for parallel parsing of Values
|
||||
static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int balance)
|
||||
{
|
||||
skipWhitespaceIfAny(*buf);
|
||||
if (buf->eof() || *buf->position() == ';')
|
||||
return false;
|
||||
bool quoted = false;
|
||||
|
||||
size_t chunk_begin_buf_count = buf->count();
|
||||
while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes))
|
||||
{
|
||||
buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end());
|
||||
if (buf->position() == buf->buffer().end())
|
||||
continue;
|
||||
if (*buf->position() == '\\')
|
||||
{
|
||||
++buf->position();
|
||||
if (!buf->eof())
|
||||
++buf->position();
|
||||
}
|
||||
else if (*buf->position() == '\'')
|
||||
{
|
||||
quoted ^= true;
|
||||
++buf->position();
|
||||
}
|
||||
else if (*buf->position() == ')')
|
||||
{
|
||||
++buf->position();
|
||||
if (!quoted)
|
||||
--balance;
|
||||
}
|
||||
else if (*buf->position() == '(')
|
||||
{
|
||||
++buf->position();
|
||||
if (!quoted)
|
||||
++balance;
|
||||
}
|
||||
}
|
||||
|
||||
if (!buf->eof() && *buf->position() == ',')
|
||||
++buf->position();
|
||||
return true;
|
||||
}
|
||||
|
||||
/// We need continuous memory containing the expression to use Lexer
|
||||
/// Note that this is both reading and tokenizing until the end of the row
|
||||
/// This is doing unnecessary work if the rest of the columns can be read with tryReadValue (which doesn't require tokens)
|
||||
/// and it's more efficient if they don't (as everything is already tokenized)
|
||||
void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize(size_t current_column_idx)
|
||||
{
|
||||
if (tokens && token_iterator &&
|
||||
/// Make sure the underlying memory hasn't changed because of next() calls in the buffer
|
||||
((*token_iterator)->begin >= buf->buffer().begin() && (*token_iterator)->begin <= buf->buffer().end()))
|
||||
{
|
||||
while ((*token_iterator)->begin < buf->position() && !(*token_iterator)->isError() && !(*token_iterator)->isEnd())
|
||||
++(*token_iterator);
|
||||
|
||||
if (!(*token_iterator)->isError() && !(*token_iterator)->isEnd())
|
||||
return;
|
||||
}
|
||||
|
||||
skipToNextRow(buf.get(), 0, 1);
|
||||
buf->makeContinuousMemoryFromCheckpointToPos();
|
||||
auto * row_end = buf->position();
|
||||
buf->rollbackToCheckpoint();
|
||||
tokens.emplace(buf->position(), row_end);
|
||||
token_iterator.emplace(*tokens, static_cast<unsigned>(context->getSettingsRef().max_parser_depth));
|
||||
auto const & first = (*token_iterator).get();
|
||||
if (first.isError() || first.isEnd())
|
||||
{
|
||||
const Block & header = getPort().getHeader();
|
||||
const IDataType & type = *header.getByPosition(current_column_idx).type;
|
||||
throw Exception(
|
||||
ErrorCodes::SYNTAX_ERROR,
|
||||
"Cannot parse expression of type {} here: {}",
|
||||
type.getName(),
|
||||
std::string_view(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())));
|
||||
}
|
||||
}
|
||||
|
||||
void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num)
|
||||
{
|
||||
tokens.reset();
|
||||
token_iterator.reset();
|
||||
assertChar('(', *buf);
|
||||
|
||||
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
|
||||
{
|
||||
skipWhitespaceIfAny(*buf);
|
||||
PeekableReadBufferCheckpoint checkpoint{*buf};
|
||||
bool read;
|
||||
bool read = false;
|
||||
|
||||
/// Parse value using fast streaming parser for literals and slow SQL parser for expressions.
|
||||
/// If there is SQL expression in some row, template of this expression will be deduced,
|
||||
@ -126,7 +208,7 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num)
|
||||
read = tryReadValue(*columns[column_idx], column_idx);
|
||||
else if (parser_type_for_column[column_idx] == ParserType::BatchTemplate)
|
||||
read = tryParseExpressionUsingTemplate(columns[column_idx], column_idx);
|
||||
else /// if (parser_type_for_column[column_idx] == ParserType::SingleExpressionEvaluation)
|
||||
else
|
||||
read = parseExpression(*columns[column_idx], column_idx);
|
||||
|
||||
if (!read)
|
||||
@ -143,9 +225,12 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num)
|
||||
|
||||
bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx)
|
||||
{
|
||||
readUntilTheEndOfRowAndReTokenize(column_idx);
|
||||
IParser::Pos start = *token_iterator;
|
||||
|
||||
/// Try to parse expression using template if one was successfully deduced while parsing the first row
|
||||
auto settings = context->getSettingsRef();
|
||||
if (templates[column_idx]->parseExpression(*buf, format_settings, settings))
|
||||
const auto & settings = context->getSettingsRef();
|
||||
if (templates[column_idx]->parseExpression(*buf, *token_iterator, format_settings, settings))
|
||||
{
|
||||
++rows_parsed_using_template[column_idx];
|
||||
return true;
|
||||
@ -166,6 +251,7 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr &
|
||||
/// Do not use this template anymore
|
||||
templates[column_idx].reset();
|
||||
buf->rollbackToCheckpoint();
|
||||
*token_iterator = start;
|
||||
|
||||
/// It will deduce new template or fallback to slow SQL parser
|
||||
return parseExpression(*column, column_idx);
|
||||
@ -295,79 +381,41 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
/// Can be used in fileSegmentationEngine for parallel parsing of Values
|
||||
static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int balance)
|
||||
{
|
||||
skipWhitespaceIfAny(*buf);
|
||||
if (buf->eof() || *buf->position() == ';')
|
||||
return false;
|
||||
bool quoted = false;
|
||||
|
||||
size_t chunk_begin_buf_count = buf->count();
|
||||
while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes))
|
||||
{
|
||||
buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end());
|
||||
if (buf->position() == buf->buffer().end())
|
||||
continue;
|
||||
if (*buf->position() == '\\')
|
||||
{
|
||||
++buf->position();
|
||||
if (!buf->eof())
|
||||
++buf->position();
|
||||
}
|
||||
else if (*buf->position() == '\'')
|
||||
{
|
||||
quoted ^= true;
|
||||
++buf->position();
|
||||
}
|
||||
else if (*buf->position() == ')')
|
||||
{
|
||||
++buf->position();
|
||||
if (!quoted)
|
||||
--balance;
|
||||
}
|
||||
else if (*buf->position() == '(')
|
||||
{
|
||||
++buf->position();
|
||||
if (!quoted)
|
||||
++balance;
|
||||
}
|
||||
}
|
||||
|
||||
if (!buf->eof() && *buf->position() == ',')
|
||||
++buf->position();
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
|
||||
{
|
||||
const Block & header = getPort().getHeader();
|
||||
const IDataType & type = *header.getByPosition(column_idx).type;
|
||||
auto settings = context->getSettingsRef();
|
||||
|
||||
/// We need continuous memory containing the expression to use Lexer
|
||||
skipToNextRow(buf.get(), 0, 1);
|
||||
buf->makeContinuousMemoryFromCheckpointToPos();
|
||||
buf->rollbackToCheckpoint();
|
||||
/// Advance the token iterator until the start of the column expression
|
||||
readUntilTheEndOfRowAndReTokenize(column_idx);
|
||||
|
||||
Expected expected;
|
||||
Tokens tokens(buf->position(), buf->buffer().end());
|
||||
IParser::Pos token_iterator(tokens, static_cast<unsigned>(settings.max_parser_depth));
|
||||
bool parsed = false;
|
||||
ASTPtr ast;
|
||||
std::optional<IParser::Pos> ti_start;
|
||||
|
||||
bool parsed = parser.parse(token_iterator, ast, expected);
|
||||
if (!(*token_iterator)->isError() && !(*token_iterator)->isEnd())
|
||||
{
|
||||
Expected expected;
|
||||
/// Keep a copy to the start of the column tokens to use if later if necessary
|
||||
ti_start = IParser::Pos(*token_iterator, static_cast<unsigned>(settings.max_parser_depth));
|
||||
|
||||
/// Consider delimiter after value (',' or ')') as part of expression
|
||||
if (column_idx + 1 != num_columns)
|
||||
parsed &= token_iterator->type == TokenType::Comma;
|
||||
else
|
||||
parsed &= token_iterator->type == TokenType::ClosingRoundBracket;
|
||||
parsed = parser.parse(*token_iterator, ast, expected);
|
||||
|
||||
/// Consider delimiter after value (',' or ')') as part of expression
|
||||
if (column_idx + 1 != num_columns)
|
||||
parsed &= (*token_iterator)->type == TokenType::Comma;
|
||||
else
|
||||
parsed &= (*token_iterator)->type == TokenType::ClosingRoundBracket;
|
||||
}
|
||||
|
||||
if (!parsed)
|
||||
throw Exception("Cannot parse expression of type " + type.getName() + " here: "
|
||||
+ String(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())),
|
||||
ErrorCodes::SYNTAX_ERROR);
|
||||
++token_iterator;
|
||||
throw Exception(
|
||||
ErrorCodes::SYNTAX_ERROR,
|
||||
"Cannot parse expression of type {} here: {}",
|
||||
type.getName(),
|
||||
std::string_view(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())));
|
||||
++(*token_iterator);
|
||||
|
||||
if (parser_type_for_column[column_idx] != ParserType::Streaming && dynamic_cast<const ASTLiteral *>(ast.get()))
|
||||
{
|
||||
@ -417,8 +465,8 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
|
||||
auto structure = templates_cache.getFromCacheOrConstruct(
|
||||
result_type,
|
||||
!result_type->isNullable() && format_settings.null_as_default,
|
||||
TokenIterator(tokens),
|
||||
token_iterator,
|
||||
*ti_start,
|
||||
*token_iterator,
|
||||
ast,
|
||||
context,
|
||||
&found_in_cache,
|
||||
@ -430,7 +478,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
|
||||
++attempts_to_deduce_template[column_idx];
|
||||
|
||||
buf->rollbackToCheckpoint();
|
||||
if (templates[column_idx]->parseExpression(*buf, format_settings, settings))
|
||||
if (templates[column_idx]->parseExpression(*buf, *ti_start, format_settings, settings))
|
||||
{
|
||||
++rows_parsed_using_template[column_idx];
|
||||
parser_type_for_column[column_idx] = ParserType::BatchTemplate;
|
||||
@ -448,7 +496,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
|
||||
else
|
||||
{
|
||||
buf->rollbackToCheckpoint();
|
||||
size_t len = const_cast<char *>(token_iterator->begin) - buf->position();
|
||||
size_t len = const_cast<char *>((*token_iterator)->begin) - buf->position();
|
||||
throw Exception("Cannot deduce template of expression: " + std::string(buf->position(), len), ErrorCodes::SYNTAX_ERROR);
|
||||
}
|
||||
}
|
||||
@ -460,7 +508,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
|
||||
throw Exception("Interpreting expressions is disabled", ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
|
||||
/// Try to evaluate single expression if other parsers don't work
|
||||
buf->position() = const_cast<char *>(token_iterator->begin);
|
||||
buf->position() = const_cast<char *>((*token_iterator)->begin);
|
||||
|
||||
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(ast, context);
|
||||
|
||||
|
@ -56,6 +56,7 @@ private:
|
||||
Chunk generate() override;
|
||||
|
||||
void readRow(MutableColumns & columns, size_t row_num);
|
||||
void readUntilTheEndOfRowAndReTokenize(size_t current_column_idx);
|
||||
|
||||
bool tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx);
|
||||
ALWAYS_INLINE inline bool tryReadValue(IColumn & column, size_t column_idx);
|
||||
@ -70,6 +71,8 @@ private:
|
||||
void readSuffix();
|
||||
|
||||
std::unique_ptr<PeekableReadBuffer> buf;
|
||||
std::optional<IParser::Pos> token_iterator{};
|
||||
std::optional<Tokens> tokens{};
|
||||
|
||||
const RowInputFormatParams params;
|
||||
|
||||
|
@ -572,20 +572,17 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
auto insert_it = constraints.end();
|
||||
auto * insert_it = constraints.end();
|
||||
constraints.emplace(insert_it, constraint_decl);
|
||||
metadata.constraints = ConstraintsDescription(constraints);
|
||||
}
|
||||
else if (type == DROP_CONSTRAINT)
|
||||
{
|
||||
auto constraints = metadata.constraints.getConstraints();
|
||||
auto erase_it = std::find_if(
|
||||
constraints.begin(),
|
||||
constraints.end(),
|
||||
[this](const ASTPtr & constraint_ast)
|
||||
{
|
||||
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
|
||||
});
|
||||
auto * erase_it = std::find_if(
|
||||
constraints.begin(),
|
||||
constraints.end(),
|
||||
[this](const ASTPtr & constraint_ast) { return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name; });
|
||||
|
||||
if (erase_it == constraints.end())
|
||||
{
|
||||
|
@ -107,7 +107,7 @@ std::unique_ptr<ComparisonGraph> ConstraintsDescription::buildGraph() const
|
||||
{
|
||||
static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" };
|
||||
|
||||
std::vector<ASTPtr> constraints_for_graph;
|
||||
ASTs constraints_for_graph;
|
||||
auto atomic_formulas = getAtomicConstraintData();
|
||||
for (const auto & atomic_formula : atomic_formulas)
|
||||
{
|
||||
@ -153,7 +153,7 @@ const std::vector<std::vector<CNFQuery::AtomicFormula>> & ConstraintsDescription
|
||||
return cnf_constraints;
|
||||
}
|
||||
|
||||
const std::vector<ASTPtr> & ConstraintsDescription::getConstraints() const
|
||||
const ASTs & ConstraintsDescription::getConstraints() const
|
||||
{
|
||||
return constraints;
|
||||
}
|
||||
@ -218,7 +218,7 @@ void ConstraintsDescription::update()
|
||||
{
|
||||
cnf_constraints.clear();
|
||||
ast_to_atom_ids.clear();
|
||||
graph = std::make_unique<ComparisonGraph>(std::vector<ASTPtr>());
|
||||
graph = std::make_unique<ComparisonGraph>(ASTs());
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -88,7 +88,7 @@ void MergeTreeIndexhypothesisMergedCondition::addConstraints(const ConstraintsDe
|
||||
/// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists
|
||||
bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const
|
||||
{
|
||||
std::vector<ASTPtr> active_atomic_formulas(atomic_constraints);
|
||||
ASTs active_atomic_formulas(atomic_constraints);
|
||||
for (const auto & hypothesis : index_to_compare_atomic_hypotheses)
|
||||
{
|
||||
active_atomic_formulas.insert(
|
||||
@ -190,7 +190,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree
|
||||
|
||||
std::unique_ptr<ComparisonGraph> MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector<bool> & values) const
|
||||
{
|
||||
std::vector<ASTPtr> active_atomic_formulas(atomic_constraints);
|
||||
ASTs active_atomic_formulas(atomic_constraints);
|
||||
for (size_t i = 0; i < values.size(); ++i)
|
||||
{
|
||||
if (values[i])
|
||||
|
@ -34,7 +34,7 @@ private:
|
||||
|
||||
std::vector<std::vector<ASTPtr>> index_to_compare_atomic_hypotheses;
|
||||
std::vector<std::vector<CNFQuery::OrGroup>> index_to_atomic_hypotheses;
|
||||
std::vector<ASTPtr> atomic_constraints;
|
||||
ASTs atomic_constraints;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -33,7 +33,7 @@ PartitionedSink::PartitionedSink(
|
||||
, context(context_)
|
||||
, sample_block(sample_block_)
|
||||
{
|
||||
std::vector<ASTPtr> arguments(1, partition_by);
|
||||
ASTs arguments(1, partition_by);
|
||||
ASTPtr partition_by_string = makeASTFunction(FunctionToString::name, std::move(arguments));
|
||||
|
||||
auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList());
|
||||
|
@ -1291,7 +1291,7 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt
|
||||
"Storage S3 requires 1 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
auto header_it = StorageURL::collectHeaders(engine_args, configuration, local_context);
|
||||
auto * header_it = StorageURL::collectHeaders(engine_args, configuration, local_context);
|
||||
if (header_it != engine_args.end())
|
||||
engine_args.erase(header_it);
|
||||
|
||||
|
@ -982,7 +982,7 @@ ASTs::iterator StorageURL::collectHeaders(
|
||||
{
|
||||
ASTs::iterator headers_it = url_function_args.end();
|
||||
|
||||
for (auto arg_it = url_function_args.begin(); arg_it != url_function_args.end(); ++arg_it)
|
||||
for (auto * arg_it = url_function_args.begin(); arg_it != url_function_args.end(); ++arg_it)
|
||||
{
|
||||
const auto * headers_ast_function = (*arg_it)->as<ASTFunction>();
|
||||
if (headers_ast_function && headers_ast_function->name == "headers")
|
||||
@ -1068,7 +1068,7 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex
|
||||
if (args.empty() || args.size() > 3)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, bad_arguments_error_message);
|
||||
|
||||
auto header_it = collectHeaders(args, configuration, local_context);
|
||||
auto * header_it = collectHeaders(args, configuration, local_context);
|
||||
if (header_it != args.end())
|
||||
args.erase(header_it);
|
||||
|
||||
|
@ -51,7 +51,7 @@ bool isValidFunction(const ASTPtr & expression, const std::function<bool(const A
|
||||
}
|
||||
|
||||
/// Extract all subfunctions of the main conjunction, but depending only on the specified columns
|
||||
bool extractFunctions(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant, std::vector<ASTPtr> & result)
|
||||
bool extractFunctions(const ASTPtr & expression, const std::function<bool(const ASTPtr &)> & is_constant, ASTs & result)
|
||||
{
|
||||
const auto * function = expression->as<ASTFunction>();
|
||||
if (function && (function->name == "and" || function->name == "indexHint"))
|
||||
@ -175,7 +175,7 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block
|
||||
};
|
||||
|
||||
/// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns.
|
||||
std::vector<ASTPtr> functions;
|
||||
ASTs functions;
|
||||
if (select.where())
|
||||
unmodified &= extractFunctions(select.where(), is_constant, functions);
|
||||
if (select.prewhere())
|
||||
|
@ -33,7 +33,7 @@ void TableFunctionDeltaLake::parseArgumentsImpl(
|
||||
if (args.empty() || args.size() > 6)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message);
|
||||
|
||||
auto header_it = StorageURL::collectHeaders(args, base_configuration, context);
|
||||
auto * header_it = StorageURL::collectHeaders(args, base_configuration, context);
|
||||
if (header_it != args.end())
|
||||
args.erase(header_it);
|
||||
|
||||
|
@ -33,7 +33,7 @@ void TableFunctionHudi::parseArgumentsImpl(
|
||||
if (args.empty() || args.size() > 6)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message);
|
||||
|
||||
auto header_it = StorageURL::collectHeaders(args, base_configuration, context);
|
||||
auto * header_it = StorageURL::collectHeaders(args, base_configuration, context);
|
||||
if (header_it != args.end())
|
||||
args.erase(header_it);
|
||||
|
||||
|
@ -45,7 +45,7 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr
|
||||
mysql_settings.connect_timeout = settings.external_storage_connect_timeout_sec;
|
||||
mysql_settings.read_write_timeout = settings.external_storage_rw_timeout_sec;
|
||||
|
||||
for (auto it = args.begin(); it != args.end(); ++it)
|
||||
for (auto * it = args.begin(); it != args.end(); ++it)
|
||||
{
|
||||
const ASTSetQuery * settings_ast = (*it)->as<ASTSetQuery>();
|
||||
if (settings_ast)
|
||||
|
@ -40,7 +40,7 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar
|
||||
if (args.empty() || args.size() > 6)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message);
|
||||
|
||||
auto header_it = StorageURL::collectHeaders(args, s3_configuration, context);
|
||||
auto * header_it = StorageURL::collectHeaders(args, s3_configuration, context);
|
||||
if (header_it != args.end())
|
||||
args.erase(header_it);
|
||||
|
||||
|
@ -71,7 +71,7 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr co
|
||||
|
||||
auto * url_function_args_expr = assert_cast<ASTExpressionList *>(args[0].get());
|
||||
auto & url_function_args = url_function_args_expr->children;
|
||||
auto headers_it = StorageURL::collectHeaders(url_function_args, configuration, context);
|
||||
auto * headers_it = StorageURL::collectHeaders(url_function_args, configuration, context);
|
||||
/// ITableFunctionFileLike cannot parse headers argument, so remove it.
|
||||
if (headers_it != url_function_args.end())
|
||||
url_function_args.erase(headers_it);
|
||||
|
File diff suppressed because it is too large
Load Diff
24
tests/queries/0_stateless/02482_value_block_assert.sql
Normal file
24
tests/queries/0_stateless/02482_value_block_assert.sql
Normal file
@ -0,0 +1,24 @@
|
||||
SET allow_suspicious_low_cardinality_types=1;
|
||||
CREATE TABLE range_key_dictionary_source_table__fuzz_323
|
||||
(
|
||||
`key` UInt256,
|
||||
`start_date` Int8,
|
||||
`end_date` LowCardinality(UInt256),
|
||||
`value` Tuple(UInt8, Array(DateTime), Decimal(9, 1), Array(Int16), Array(UInt8)),
|
||||
`value_nullable` UUID
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
INSERT INTO range_key_dictionary_source_table__fuzz_323 FORMAT Values
|
||||
(1, toDate('2019-05-20'), toDate('2019-05-20'), 'First', 'First'); -- { clientError CANNOT_PARSE_INPUT_ASSERTION_FAILED }
|
||||
|
||||
|
||||
CREATE TABLE complex_key_dictionary_source_table__fuzz_267
|
||||
(
|
||||
`id` Decimal(38, 30),
|
||||
`id_key` Array(UUID),
|
||||
`value` Array(Nullable(DateTime64(3))),
|
||||
`value_nullable` Nullable(UUID)
|
||||
)
|
||||
ENGINE = TinyLog;
|
||||
INSERT INTO complex_key_dictionary_source_table__fuzz_267 FORMAT Values
|
||||
(1, 'key', 'First', 'First'); -- { clientError CANNOT_READ_ARRAY_FROM_TEXT }
|
@ -0,0 +1,3 @@
|
||||
1
|
||||
1
|
||||
1
|
26
tests/queries/0_stateless/02482_value_block_parsing.sh
Executable file
26
tests/queries/0_stateless/02482_value_block_parsing.sh
Executable file
@ -0,0 +1,26 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="
|
||||
CREATE TABLE simple_key_dictionary_source_table__fuzz_48
|
||||
(
|
||||
id Nullable(Int8),
|
||||
value Array(Date),
|
||||
value_nullable UUID
|
||||
)
|
||||
ENGINE = TinyLog;"
|
||||
|
||||
echo "INSERT INTO simple_key_dictionary_source_table__fuzz_48 FORMAT Values (null, [], '61f0c404-5cb3-11e7-907b-a6006ad3dba0')
|
||||
( -- Bu " | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" --data-binary @- -v 2>&1 | grep -c 'X-ClickHouse-Exception-Code: 62'
|
||||
|
||||
|
||||
echo "INSERT INTO simple_key_dictionary_source_table__fuzz_48 FORMAT Values
|
||||
(!Invalid" | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" --data-binary @- -v 2>&1 | grep -c 'X-ClickHouse-Exception-Code: 62'
|
||||
|
||||
echo "INSERT INTO simple_key_dictionary_source_table__fuzz_48 FORMAT Values (null, [], '61f0c404-5cb3-11e7-907b-a6006ad3dba0')
|
||||
,(null, [], '61f0c404-5cb3-11e7-907b-a6006ad3dba0'),
|
||||
(!!!!!!3adas
|
||||
)" | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" --data-binary @- -v 2>&1 | grep -c 'X-ClickHouse-Exception-Code: 62'
|
Loading…
Reference in New Issue
Block a user