mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Get rid of buggy code (that coincidently was useless too) [#METR-2944].
This commit is contained in:
parent
322e5031bd
commit
4d65dd3b37
@ -10,7 +10,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Применение функции или оператора
|
||||
/** AST for function application or operator.
|
||||
*/
|
||||
class ASTFunction : public ASTWithAlias
|
||||
{
|
||||
@ -25,41 +25,26 @@ public:
|
||||
ARRAY_JOIN,
|
||||
};
|
||||
|
||||
/// имя функции
|
||||
String name;
|
||||
/// аргументы
|
||||
ASTPtr arguments;
|
||||
/// параметры - для параметрических агрегатных функций. Пример: quantile(0.9)(x) - то, что в первых скобках - параметры.
|
||||
/// parameters - for parametric aggregate function. Example: quantile(0.9)(x) - what in first parens are 'parameters'.
|
||||
ASTPtr parameters;
|
||||
|
||||
FunctionKind kind{UNKNOWN};
|
||||
|
||||
enum class Genus
|
||||
{
|
||||
ORDINARY = 0,
|
||||
CASE_WITH_EXPR,
|
||||
CASE_WITHOUT_EXPR,
|
||||
CASE_ARRAY
|
||||
};
|
||||
|
||||
Genus genus{Genus::ORDINARY};
|
||||
|
||||
public:
|
||||
ASTFunction() = default;
|
||||
ASTFunction(const StringRange range_) : ASTWithAlias(range_) {}
|
||||
|
||||
String getColumnName() const override;
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
/** Get text identifying the AST node. */
|
||||
String getID() const override;
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
protected:
|
||||
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
|
||||
private:
|
||||
void formatCase(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
|
||||
};
|
||||
|
||||
|
||||
|
@ -129,37 +129,6 @@ bool functionIsInOrGlobalInOperator(const String & name)
|
||||
return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn";
|
||||
}
|
||||
|
||||
/// Create a function uniquely identified by the first two parameters. If this
|
||||
/// function actually is a CASE expression, record this information.
|
||||
FunctionPtr getFunctionFromFactory(const String & name, const ASTFunction::Genus genus, const Context & context)
|
||||
{
|
||||
FunctionPtr function = FunctionFactory::instance().get(name, context);
|
||||
|
||||
if (genus == ASTFunction::Genus::CASE_WITH_EXPR)
|
||||
{
|
||||
FunctionTransform * fun_transform = typeid_cast<FunctionTransform *>(&*function);
|
||||
if (fun_transform == nullptr)
|
||||
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
||||
fun_transform->setCaseMode();
|
||||
}
|
||||
else if (genus == ASTFunction::Genus::CASE_WITHOUT_EXPR)
|
||||
{
|
||||
FunctionMultiIf * fun_multi_if = typeid_cast<FunctionMultiIf *>(&*function);
|
||||
if (fun_multi_if == nullptr)
|
||||
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
||||
fun_multi_if->setCaseMode();
|
||||
}
|
||||
else if (genus == ASTFunction::Genus::CASE_ARRAY)
|
||||
{
|
||||
FunctionArray * fun_array = typeid_cast<FunctionArray *>(&*function);
|
||||
if (fun_array == nullptr)
|
||||
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
|
||||
fun_array->setCaseMode();
|
||||
}
|
||||
|
||||
return function;
|
||||
}
|
||||
|
||||
void removeDuplicateColumns(NamesAndTypesList & columns)
|
||||
{
|
||||
std::set<String> names;
|
||||
@ -1753,7 +1722,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
|
||||
return;
|
||||
}
|
||||
|
||||
const FunctionPtr & function = getFunctionFromFactory(node->name, node->genus, context);
|
||||
const FunctionPtr & function = FunctionFactory::instance().get(node->name, context);
|
||||
|
||||
Names argument_names;
|
||||
DataTypes argument_types;
|
||||
|
@ -66,13 +66,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
||||
nested_need_parens.need_parens = true;
|
||||
nested_dont_need_parens.need_parens = false;
|
||||
|
||||
if ((genus == Genus::CASE_WITH_EXPR) || (genus == Genus::CASE_WITHOUT_EXPR))
|
||||
{
|
||||
formatCase(settings, state, frame);
|
||||
return;
|
||||
}
|
||||
|
||||
/// Стоит ли записать эту функцию в виде оператора?
|
||||
/// Should this function to be written as operator?
|
||||
bool written = false;
|
||||
if (arguments && !parameters)
|
||||
{
|
||||
@ -264,84 +258,4 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
||||
}
|
||||
}
|
||||
|
||||
void ASTFunction::formatCase(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
static constexpr auto s_case = "CASE";
|
||||
static constexpr auto s_when = "WHEN";
|
||||
static constexpr auto s_then = "THEN";
|
||||
static constexpr auto s_else = "ELSE";
|
||||
static constexpr auto s_end = "END";
|
||||
static constexpr auto s_ws = " ";
|
||||
|
||||
const ASTExpressionList * expr_list = static_cast<const ASTExpressionList *>(&*arguments);
|
||||
const ASTs & args = expr_list->children;
|
||||
|
||||
frame.need_parens = false;
|
||||
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
|
||||
std::string indent_str2 = settings.one_line ? "" : std::string(4 * (frame.indent + 1), ' ');
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << s_case << s_ws
|
||||
<< (settings.hilite ? hilite_none : "");
|
||||
|
||||
if (genus == Genus::CASE_WITH_EXPR)
|
||||
{
|
||||
args[0]->formatImpl(settings, state, frame);
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
|
||||
const ASTFunction * src_array_function = static_cast<const ASTFunction *>(&*args[1]);
|
||||
const ASTExpressionList * src_expr_list = static_cast<const ASTExpressionList *>(&*src_array_function->arguments);
|
||||
|
||||
const ASTFunction * dst_array_function = static_cast<const ASTFunction *>(&*args[2]);
|
||||
const ASTExpressionList * dst_expr_list = static_cast<const ASTExpressionList *>(&*dst_array_function->arguments);
|
||||
|
||||
size_t size = src_expr_list->children.size();
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "")
|
||||
<< indent_str2 << s_when << s_ws;
|
||||
src_expr_list->children[i]->formatImpl(settings, state, frame);
|
||||
settings.ostr << s_ws;
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << s_then << s_ws;
|
||||
settings.ostr << hilite_none;
|
||||
dst_expr_list->children[i]->formatImpl(settings, state, frame);
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
}
|
||||
}
|
||||
else if (genus == Genus::CASE_WITHOUT_EXPR)
|
||||
{
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
|
||||
for (size_t i = 0; i < (args.size() - 1); ++i)
|
||||
{
|
||||
if ((i % 2) == 0)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "")
|
||||
<< indent_str2 << s_when << s_ws;
|
||||
args[i]->formatImpl(settings, state, frame);
|
||||
settings.ostr << " ";
|
||||
}
|
||||
else
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << s_then << s_ws;
|
||||
settings.ostr << hilite_none;
|
||||
args[i]->formatImpl(settings, state, frame);
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception{"Invalid function genus", ErrorCodes::INVALID_FUNCTION_GENUS};
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str2
|
||||
<< s_else << s_ws;
|
||||
settings.ostr << hilite_none;
|
||||
args.back()->formatImpl(settings, state, frame);
|
||||
settings.ostr << settings.nl_or_ws;
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << s_end;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -113,13 +113,11 @@ bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
|
||||
|
||||
auto src_array_function = std::make_shared<ASTFunction>(StringRange{begin, pos});
|
||||
src_array_function->name = "array";
|
||||
src_array_function->genus = ASTFunction::Genus::CASE_ARRAY;
|
||||
src_array_function->arguments = src_expr_list;
|
||||
src_array_function->children.push_back(src_array_function->arguments);
|
||||
|
||||
auto dst_array_function = std::make_shared<ASTFunction>(StringRange{begin, pos});
|
||||
dst_array_function->name = "array";
|
||||
dst_array_function->genus = ASTFunction::Genus::CASE_ARRAY;
|
||||
dst_array_function->arguments = dst_expr_list;
|
||||
dst_array_function->children.push_back(dst_array_function->arguments);
|
||||
|
||||
@ -131,7 +129,6 @@ bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
|
||||
|
||||
auto function = std::make_shared<ASTFunction>(StringRange{begin, pos});
|
||||
function->name = "transform";
|
||||
function->genus = ASTFunction::Genus::CASE_WITH_EXPR;
|
||||
function->arguments = function_args;
|
||||
function->children.push_back(function->arguments);
|
||||
|
||||
@ -148,7 +145,6 @@ bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_p
|
||||
|
||||
auto function = std::make_shared<ASTFunction>(StringRange{begin, pos});
|
||||
function->name = "multiIf";
|
||||
function->genus = ASTFunction::Genus::CASE_WITHOUT_EXPR;
|
||||
function->arguments = function_args;
|
||||
function->children.push_back(function->arguments);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user