提交 eb057981 编写于 作者: V Vitaliy Lyudvichenko

Add SYSTEM queries parser. [#CLICKHOUSE-2931]

上级 fd63277b
......@@ -32,6 +32,9 @@
#include <Interpreters/InterpreterFactory.h>
#include <Common/typeid_cast.h>
#include <Parsers/ASTSystemQuery.h>
#include "InterpreterSystemQuery.h"
namespace DB
{
......@@ -123,6 +126,11 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
{
return std::make_unique<InterpreterKillQueryQuery>(query, context);
}
else if (typeid_cast<ASTSystemQuery *>(query.get()))
{
throwIfReadOnly(context);
return std::make_unique<InterpreterSystemQuery>(query, context);
}
else
throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
}
......
#include "InterpreterSystemQuery.h"
namespace DB
{
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_) {}
BlockIO InterpreterSystemQuery::execute()
{
return BlockIO();
}
}
\ No newline at end of file
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class Context;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
/** Return list of currently executing queries.
*/
class InterpreterSystemQuery : public IInterpreter
{
public:
InterpreterSystemQuery(const ASTPtr & query_ptr_, Context & context_);
BlockIO execute() override;
private:
ASTPtr query_ptr;
Context & context;
};
}
......@@ -4,6 +4,7 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ExpressionElementParsers.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionAnalyzer.h>
......@@ -62,10 +63,30 @@ ASTPtr evaluateConstantExpressionAsLiteral(ASTPtr & node, const Context & contex
ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(ASTPtr & node, const Context & context)
{
if (const ASTIdentifier * id = typeid_cast<const ASTIdentifier *>(node.get()))
if (auto id = typeid_cast<const ASTIdentifier *>(node.get()))
return std::make_shared<ASTLiteral>(node->range, Field(id->name));
return evaluateConstantExpressionAsLiteral(node, context);
}
bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, String & result)
{
IParser::Pos begin = pos;
ASTPtr res;
if (!ParserIdentifier().parse(pos, res, expected))
{
pos = begin;
if (!ParserStringLiteral().parse(pos, res, expected))
return false;
result = typeid_cast<const ASTLiteral &>(*res).value.safeGet<String>();
}
else
result = typeid_cast<const ASTIdentifier &>(*res).name;
return true;
}
}
......@@ -2,6 +2,7 @@
#include <memory>
#include <Core/Field.h>
#include <Parsers/IParser.h>
namespace DB
......@@ -11,6 +12,7 @@ class IAST;
class Context;
class IDataType;
/** Evaluate constant expression and its type.
* Used in rare cases - for elements of set for IN, for data to INSERT.
* Quite suboptimal.
......@@ -31,4 +33,8 @@ std::shared_ptr<IAST> evaluateConstantExpressionAsLiteral(std::shared_ptr<IAST>
*/
std::shared_ptr<IAST> evaluateConstantExpressionOrIdentifierAsLiteral(std::shared_ptr<IAST> & node, const Context & context);
/** Parses a name of an object which could be written in 3 forms:
* name, `name` or 'name' */
bool parseIdentifierOrStringLiteral(IParser::Pos & pos, Expected & expected, String & result);
}
......@@ -5,6 +5,7 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Common/typeid_cast.h>
#include <Interpreters/evaluateConstantExpression.h>
namespace DB
......@@ -21,21 +22,7 @@ bool ASTQueryWithOnCluster::parse(Pos & pos, std::string & cluster_str, Expected
if (!ParserKeyword{"CLUSTER"}.ignore(pos, expected))
return false;
Pos begin = pos;
ASTPtr res;
if (!ParserIdentifier().parse(pos, res, expected))
{
pos = begin;
if (!ParserStringLiteral().parse(pos, res, expected))
return false;
else
cluster_str = typeid_cast<const ASTLiteral &>(*res).value.safeGet<String>();
}
else
cluster_str = typeid_cast<const ASTIdentifier &>(*res).name;
return true;
return parseIdentifierOrStringLiteral(pos, expected, cluster_str);
}
......
......@@ -24,7 +24,7 @@ public:
Changes changes;
ASTSetQuery() = default;
ASTSetQuery(const StringRange range_) : IAST(range_) {}
explicit ASTSetQuery(const StringRange range_) : IAST(range_) {}
/** Get the text that identifies this element. */
String getID() const override { return "Set"; };
......
#include <Parsers/IAST.h>
#include <Parsers/ASTSystemQuery.h>
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
}
namespace DB
{
const char * ASTSystemQuery::typeToString(Type type)
{
switch (type)
{
case Type::SHUTDOWN:
return "SHUTDOWN";
case Type::KILL:
return "KILL";
case Type::DROP_DNS_CACHE:
return "DROP DNS CACHE";
case Type::DROP_MARK_CACHE:
return "DROP MARK CACHE";
case Type::DROP_UNCOMPRESSED_CACHE:
return "DROP UNCOMPRESSED CACHE";
case Type::STOP_LISTEN_QUERIES:
return "STOP LISTEN QUERIES";
case Type::START_LISTEN_QUERIES:
return "START LISTEN QUERIES";
case Type::RESTART_REPLICAS:
return "RESTART REPLICAS";
case Type::SYNC_REPLICA:
return "SYNC REPLICA";
case Type::RELOAD_DICTIONARY:
return "RELOAD DICTIONARY";
case Type::RELOAD_DICTIONARIES:
return "RELOAD DICTIONARIES";
case Type::STOP_MERGES:
return "STOP MERGES";
case Type::START_MERGES:
return "START MERGES";
case Type::STOP_REPLICATION_QUEUES:
return "STOP REPLICATION QUEUES";
case Type::START_REPLICATION_QUEUES:
return "START REPLICATION QUEUES";
default:
throw Exception("Unknown SYSTEM query command", ErrorCodes::BAD_TYPE_OF_FIELD);
}
}
}
\ No newline at end of file
#pragma once
#include <Parsers/IAST.h>
#include <AggregateFunctions/AggregateFunctionCount.h>
namespace DB
{
class ASTSystemQuery : public IAST
{
public:
enum class Type
{
_UNKNOWN,
SHUTDOWN,
KILL,
DROP_DNS_CACHE,
DROP_MARK_CACHE,
DROP_UNCOMPRESSED_CACHE,
STOP_LISTEN_QUERIES,
START_LISTEN_QUERIES,
RESTART_REPLICAS,
SYNC_REPLICA,
RELOAD_DICTIONARY,
RELOAD_DICTIONARIES,
STOP_MERGES,
START_MERGES,
STOP_REPLICATION_QUEUES,
START_REPLICATION_QUEUES,
_END
};
static const char * typeToString(Type type);
Type type = Type::_UNKNOWN;
String target_dictionary;
//String target_replica;
ASTSystemQuery() = default;
explicit ASTSystemQuery(const StringRange range) : IAST(range) {}
String getID() const override { return "SYSTEM"; };
ASTPtr clone() const override { return std::make_shared<ASTSystemQuery>(*this); }
~ASTSystemQuery() override = default;
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM " << (settings.hilite ? hilite_none : "");
}
};
}
\ No newline at end of file
......@@ -8,6 +8,7 @@
#include <Parsers/ParserUseQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ParserAlterQuery.h>
#include "ParserSystemQuery.h"
namespace DB
......@@ -22,12 +23,14 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserUseQuery use_p;
ParserSetQuery set_p;
ParserOptimizeQuery optimize_p;
ParserSystemQuery system_p;
bool res = query_with_output_p.parse(pos, node, expected)
|| insert_p.parse(pos, node, expected)
|| use_p.parse(pos, node, expected)
|| set_p.parse(pos, node, expected)
|| optimize_p.parse(pos, node, expected);
|| optimize_p.parse(pos, node, expected)
|| system_p.parse(pos, node, expected);
return res;
}
......
......@@ -13,11 +13,11 @@ namespace DB
class ParserSetQuery : public IParserBase
{
public:
ParserSetQuery(bool parse_only_internals_ = false) : parse_only_internals(parse_only_internals_) {}
explicit ParserSetQuery(bool parse_only_internals_ = false) : parse_only_internals(parse_only_internals_) {}
protected:
const char * getName() const { return "SET query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
const char * getName() const override { return "SET query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
/// Parse the list `name = value` pairs, without SET [GLOBAL].
bool parse_only_internals;
......
#include <Parsers/ParserSystemQuery.h>
#include <Parsers/ASTSystemQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Interpreters/evaluateConstantExpression.h>
#include "ASTSystemQuery.h"
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
namespace DB
{
bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
if (!ParserKeyword{"SYSTEM"}.ignore(pos))
return false;
using Type = ASTSystemQuery::Type;
auto res = std::make_shared<ASTSystemQuery>();
bool found = false;
for (int i = static_cast<int>(Type::SHUTDOWN); i < static_cast<int>(Type::_END); ++i)
{
Type t = static_cast<Type>(i);
if (ParserKeyword{ASTSystemQuery::typeToString(t)}.ignore(pos))
{
res->type = t;
found = true;
}
}
if (!found)
return false;
if (res->type == Type::RELOAD_DICTIONARY)
{
if (!parseIdentifierOrStringLiteral(pos, expected, res->target_dictionary))
return false;
}
else if (res->type == Type::SYNC_REPLICA)
{
throw Exception("SYNC REPLICA is not supported yet", ErrorCodes::NOT_IMPLEMENTED);
}
node = std::move(res);
return true;
}
}
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
class ParserSystemQuery : public IParserBase
{
public:
ParserSystemQuery() = default;
protected:
const char * getName() const override { return "SYSTEM query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}
\ No newline at end of file
#!/bin/bash
set -e
cli="clickhouse-client -q "
$cli "SYSTEM SHUTDOWN"
$cli "SYSTEM KILL"
$cli "SYSTEM DROP DNS CACHE"
$cli "SYSTEM DROP MARK CACHE"
$cli "SYSTEM DROP UNCOMPRESSED CACHE"
$cli "SYSTEM STOP LISTEN QUERIES"
$cli "SYSTEM START LISTEN QUERIES"
$cli "SYSTEM RESTART REPLICAS"
#$cli "SYSTEM SYNC REPLICA test"
$cli "SYSTEM RELOAD DICTIONARY dict"
$cli "SYSTEM RELOAD DICTIONARIES"
$cli "SYSTEM STOP MERGES"
$cli "SYSTEM START MERGES"
$cli "SYSTEM STOP REPLICATION QUEUES"
$cli "SYSTEM START REPLICATION QUEUES"
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册