parser WITH FILL

some interpreter changes
This commit is contained in:
dmitriiut 2019-04-21 06:36:59 +03:00
parent 2e0374fbd4
commit 8a7853b0a8
8 changed files with 117 additions and 8 deletions

View File

@ -427,6 +427,7 @@ namespace ErrorCodes
extern const int BAD_TTL_EXPRESSION = 450; extern const int BAD_TTL_EXPRESSION = 450;
extern const int BAD_TTL_FILE = 451; extern const int BAD_TTL_FILE = 451;
extern const int WITH_TIES_WITHOUT_ORDER_BY = 452; extern const int WITH_TIES_WITHOUT_ORDER_BY = 452;
extern const int INVALID_WITH_FILL_EXPRESSION = 453;
extern const int KEEPER_EXCEPTION = 999; extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000; extern const int POCO_EXCEPTION = 1000;

View File

@ -20,12 +20,25 @@ struct SortColumnDescription
int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less. int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less.
/// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite. /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite.
std::shared_ptr<Collator> collator; /// Collator for locale-specific comparison of strings std::shared_ptr<Collator> collator; /// Collator for locale-specific comparison of strings
bool with_fill; /// If true, all missed values in range [FROM, TO] will be filled
/// Range [FROM, TO] respects sorting direction
double fill_from; /// Value >= FROM
double fill_to; /// Value + STEP <= TO
double fill_step; /// Default = 1
SortColumnDescription(size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr<Collator> & collator_ = nullptr) SortColumnDescription(
: column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} size_t column_number_, int direction_, int nulls_direction_,
const std::shared_ptr<Collator> & collator_ = nullptr, bool with_fill_ = false,
double fill_from_ = 0, double fill_to_ = 0, double fill_step_ = 0)
: column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_)
, with_fill(with_fill_), fill_from(fill_from_), fill_to(fill_to_), fill_step(fill_step_) {}
SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr<Collator> & collator_ = nullptr) SortColumnDescription(
: column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} const std::string & column_name_, int direction_, int nulls_direction_,
const std::shared_ptr<Collator> & collator_ = nullptr, bool with_fill_ = false,
double fill_from_ = 0, double fill_to_ = 0, double fill_step_ = 0)
: column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_)
, collator(collator_), with_fill(with_fill_), fill_from(fill_from_), fill_to(fill_to_), fill_step(fill_step_) {}
bool operator == (const SortColumnDescription & other) const bool operator == (const SortColumnDescription & other) const
{ {

View File

@ -74,6 +74,7 @@ namespace ErrorCodes
extern const int PARAMETER_OUT_OF_BOUND; extern const int PARAMETER_OUT_OF_BOUND;
extern const int ARGUMENT_OUT_OF_BOUND; extern const int ARGUMENT_OUT_OF_BOUND;
extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_LIMIT_EXPRESSION;
extern const int INVALID_WITH_FILL_EXPRESSION;
} }
namespace namespace
@ -888,6 +889,34 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c
return 0; return 0;
} }
static Float64 getWithFillFloatValue(const ASTPtr & node, const Context & context)
{
const auto & [field, type] = evaluateConstantExpression(node, context);
if (!isNumber(type))
throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
Field converted = convertFieldToType(field, DataTypeFloat64());
if (converted.isNull())
throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of WITH FILL expression is not representable as Float64", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
return converted.safeGet<UInt64>();
}
static std::tuple<Float64, Float64, Float64> getWithFillParameters(const ASTOrderByElement & node, const Context & context)
{
Float64 fill_from = 0;
Float64 fill_to = 0;
Float64 fill_step = 1;
if (node.fill_from)
fill_from = getWithFillFloatValue(node.fill_from, context);
if (node.fill_to)
fill_to = getWithFillFloatValue(node.fill_to, context);
if (node.fill_step)
fill_step = getWithFillFloatValue(node.fill_step, context);
return {fill_from, fill_to, fill_step};
}
void InterpreterSelectQuery::executeFetchColumns( void InterpreterSelectQuery::executeFetchColumns(
QueryProcessingStage::Enum processing_stage, Pipeline & pipeline, QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
@ -1412,7 +1441,7 @@ void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const Expres
} }
static SortDescription getSortDescription(const ASTSelectQuery & query) SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query)
{ {
SortDescription order_descr; SortDescription order_descr;
order_descr.reserve(query.orderBy()->children.size()); order_descr.reserve(query.orderBy()->children.size());
@ -1425,6 +1454,13 @@ static SortDescription getSortDescription(const ASTSelectQuery & query)
if (order_by_elem.collation) if (order_by_elem.collation)
collator = std::make_shared<Collator>(order_by_elem.collation->as<ASTLiteral &>().value.get<String>()); collator = std::make_shared<Collator>(order_by_elem.collation->as<ASTLiteral &>().value.get<String>());
if (order_by_elem.with_fill)
{
auto[fill_from, fill_to, fill_step] = getWithFillParameters(order_by_elem, context);
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator,
true, fill_from, fill_to, fill_step);
}
else
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator);
} }

View File

@ -208,6 +208,9 @@ private:
*/ */
void initSettings(); void initSettings();
/// gets ORDER BY description
SortDescription getSortDescription(const ASTSelectQuery & query);
const SelectQueryOptions options; const SelectQueryOptions options;
ASTPtr query_ptr; ASTPtr query_ptr;
Context context; Context context;

View File

@ -25,6 +25,26 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState
settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : ""); settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "");
collation->formatImpl(settings, state, frame); collation->formatImpl(settings, state, frame);
} }
if (with_fill)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL " << (settings.hilite ? hilite_none : "");
if (fill_from)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "");
fill_from->formatImpl(settings, state, frame);
}
if (fill_to)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "");
fill_to->formatImpl(settings, state, frame);
}
if (fill_step)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : "");
fill_step->formatImpl(settings, state, frame);
}
}
} }
} }

View File

@ -15,15 +15,25 @@ public:
int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST.
bool nulls_direction_was_explicitly_specified; bool nulls_direction_was_explicitly_specified;
bool with_fill;
ASTPtr fill_from;
ASTPtr fill_to;
ASTPtr fill_step;
/** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */
ASTPtr collation; ASTPtr collation;
ASTOrderByElement( ASTOrderByElement(
const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_, ASTPtr & collation_) const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_,
ASTPtr & collation_, const bool with_fill_, ASTPtr & fill_from_, ASTPtr & fill_to_, ASTPtr & fill_step_)
: direction(direction_) : direction(direction_)
, nulls_direction(nulls_direction_) , nulls_direction(nulls_direction_)
, nulls_direction_was_explicitly_specified(nulls_direction_was_explicitly_specified_) , nulls_direction_was_explicitly_specified(nulls_direction_was_explicitly_specified_)
, collation(collation_) , collation(collation_)
, with_fill(with_fill_)
, fill_from(fill_from_)
, fill_to(fill_to_)
, fill_step(fill_step_)
{ {
} }

View File

@ -1283,7 +1283,12 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
ParserKeyword first("FIRST"); ParserKeyword first("FIRST");
ParserKeyword last("LAST"); ParserKeyword last("LAST");
ParserKeyword collate("COLLATE"); ParserKeyword collate("COLLATE");
ParserKeyword with_fill("WITH FILL");
ParserKeyword from("FROM");
ParserKeyword to("TO");
ParserKeyword step("STEP");
ParserStringLiteral collate_locale_parser; ParserStringLiteral collate_locale_parser;
ParserExpressionWithOptionalAlias exp_parser(false);
ASTPtr expr_elem; ASTPtr expr_elem;
if (!elem_p.parse(pos, expr_elem, expected)) if (!elem_p.parse(pos, expr_elem, expected))
@ -1318,7 +1323,27 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return false; return false;
} }
node = std::make_shared<ASTOrderByElement>(direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node); /// WITH FILL [FROM x] [TO y] [STEP z]
bool with_fill_val = false;
ASTPtr fill_from;
ASTPtr fill_to;
ASTPtr fill_step;
if (with_fill.ignore(pos))
{
with_fill_val = true;
if (from.ignore(pos) && !exp_parser.parse(pos, fill_from, expected))
return false;
if (to.ignore(pos) && !exp_parser.parse(pos, fill_to, expected))
return false;
if (step.ignore(pos) && !exp_parser.parse(pos, fill_step, expected))
return false;
}
node = std::make_shared<ASTOrderByElement>(
direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node,
with_fill, fill_from, fill_to, fill_step);
node->children.push_back(expr_elem); node->children.push_back(expr_elem);
if (locale_node) if (locale_node)
node->children.push_back(locale_node); node->children.push_back(locale_node);

View File

@ -273,6 +273,7 @@ protected:
/** Element of ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified /** Element of ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified
* and optionally, NULLS LAST|FIRST * and optionally, NULLS LAST|FIRST
* and optionally, COLLATE 'locale'. * and optionally, COLLATE 'locale'.
* and optionally, WITH FILL [FROM x] [TO y] [STEP z]
*/ */
class ParserOrderByElement : public IParserBase class ParserOrderByElement : public IParserBase
{ {