mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #11832 from ClickHouse/mysql-vars
Added support for MySQL style global variables syntax
This commit is contained in:
commit
5decc73b5d
@ -398,6 +398,7 @@ private:
|
|||||||
{ TokenType::GreaterOrEquals, Replxx::Color::INTENSE },
|
{ TokenType::GreaterOrEquals, Replxx::Color::INTENSE },
|
||||||
{ TokenType::Concatenation, Replxx::Color::INTENSE },
|
{ TokenType::Concatenation, Replxx::Color::INTENSE },
|
||||||
{ TokenType::At, Replxx::Color::INTENSE },
|
{ TokenType::At, Replxx::Color::INTENSE },
|
||||||
|
{ TokenType::DoubleAt, Replxx::Color::MAGENTA },
|
||||||
|
|
||||||
{ TokenType::EndOfStream, Replxx::Color::DEFAULT },
|
{ TokenType::EndOfStream, Replxx::Color::DEFAULT },
|
||||||
|
|
||||||
|
@ -11,6 +11,8 @@ FunctionOverloadResolverImplPtr CastOverloadResolver::create(const Context & con
|
|||||||
return createImpl(context.getSettingsRef().cast_keep_nullable);
|
return createImpl(context.getSettingsRef().cast_keep_nullable);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void registerFunctionFixedString(FunctionFactory & factory);
|
||||||
|
|
||||||
void registerFunctionsConversion(FunctionFactory & factory)
|
void registerFunctionsConversion(FunctionFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerFunction<FunctionToUInt8>();
|
factory.registerFunction<FunctionToUInt8>();
|
||||||
@ -33,7 +35,8 @@ void registerFunctionsConversion(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionToDateTime64>();
|
factory.registerFunction<FunctionToDateTime64>();
|
||||||
factory.registerFunction<FunctionToUUID>();
|
factory.registerFunction<FunctionToUUID>();
|
||||||
factory.registerFunction<FunctionToString>();
|
factory.registerFunction<FunctionToString>();
|
||||||
factory.registerFunction<FunctionToFixedString>();
|
|
||||||
|
registerFunctionFixedString(factory);
|
||||||
|
|
||||||
factory.registerFunction<FunctionToUnixTimestamp>();
|
factory.registerFunction<FunctionToUnixTimestamp>();
|
||||||
factory.registerFunction<CastOverloadResolver>(FunctionFactory::CaseInsensitive);
|
factory.registerFunction<CastOverloadResolver>(FunctionFactory::CaseInsensitive);
|
||||||
|
@ -41,6 +41,7 @@
|
|||||||
#include <Functions/DateTimeTransforms.h>
|
#include <Functions/DateTimeTransforms.h>
|
||||||
#include <DataTypes/DataTypeLowCardinality.h>
|
#include <DataTypes/DataTypeLowCardinality.h>
|
||||||
#include <Columns/ColumnLowCardinality.h>
|
#include <Columns/ColumnLowCardinality.h>
|
||||||
|
#include <Functions/toFixedString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -58,7 +59,6 @@ namespace ErrorCodes
|
|||||||
extern const int CANNOT_PARSE_DATETIME;
|
extern const int CANNOT_PARSE_DATETIME;
|
||||||
extern const int CANNOT_PARSE_TEXT;
|
extern const int CANNOT_PARSE_TEXT;
|
||||||
extern const int CANNOT_PARSE_UUID;
|
extern const int CANNOT_PARSE_UUID;
|
||||||
extern const int TOO_LARGE_STRING_SIZE;
|
|
||||||
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int LOGICAL_ERROR;
|
||||||
extern const int TYPE_MISMATCH;
|
extern const int TYPE_MISMATCH;
|
||||||
@ -1262,94 +1262,6 @@ public:
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
/** Conversion to fixed string is implemented only for strings.
|
|
||||||
*/
|
|
||||||
class FunctionToFixedString : public IFunction
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
static constexpr auto name = "toFixedString";
|
|
||||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToFixedString>(); }
|
|
||||||
static FunctionPtr create() { return std::make_shared<FunctionToFixedString>(); }
|
|
||||||
|
|
||||||
String getName() const override
|
|
||||||
{
|
|
||||||
return name;
|
|
||||||
}
|
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 2; }
|
|
||||||
bool isInjective(const Block &) const override { return true; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
|
||||||
{
|
|
||||||
if (!isUnsignedInteger(arguments[1].type))
|
|
||||||
throw Exception("Second argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
if (!arguments[1].column)
|
|
||||||
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
if (!isStringOrFixedString(arguments[0].type))
|
|
||||||
throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED);
|
|
||||||
|
|
||||||
const size_t n = arguments[1].column->getUInt(0);
|
|
||||||
return std::make_shared<DataTypeFixedString>(n);
|
|
||||||
}
|
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
|
||||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
|
||||||
{
|
|
||||||
const auto n = block.getByPosition(arguments[1]).column->getUInt(0);
|
|
||||||
return executeForN(block, arguments, result, n);
|
|
||||||
}
|
|
||||||
|
|
||||||
static void executeForN(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
|
|
||||||
{
|
|
||||||
const auto & column = block.getByPosition(arguments[0]).column;
|
|
||||||
|
|
||||||
if (const auto column_string = checkAndGetColumn<ColumnString>(column.get()))
|
|
||||||
{
|
|
||||||
auto column_fixed = ColumnFixedString::create(n);
|
|
||||||
|
|
||||||
auto & out_chars = column_fixed->getChars();
|
|
||||||
const auto & in_chars = column_string->getChars();
|
|
||||||
const auto & in_offsets = column_string->getOffsets();
|
|
||||||
|
|
||||||
out_chars.resize_fill(in_offsets.size() * n);
|
|
||||||
|
|
||||||
for (size_t i = 0; i < in_offsets.size(); ++i)
|
|
||||||
{
|
|
||||||
const size_t off = i ? in_offsets[i - 1] : 0;
|
|
||||||
const size_t len = in_offsets[i] - off - 1;
|
|
||||||
if (len > n)
|
|
||||||
throw Exception("String too long for type FixedString(" + toString(n) + ")",
|
|
||||||
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
|
||||||
memcpy(&out_chars[i * n], &in_chars[off], len);
|
|
||||||
}
|
|
||||||
|
|
||||||
block.getByPosition(result).column = std::move(column_fixed);
|
|
||||||
}
|
|
||||||
else if (const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(column.get()))
|
|
||||||
{
|
|
||||||
const auto src_n = column_fixed_string->getN();
|
|
||||||
if (src_n > n)
|
|
||||||
throw Exception{"String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE};
|
|
||||||
|
|
||||||
auto column_fixed = ColumnFixedString::create(n);
|
|
||||||
|
|
||||||
auto & out_chars = column_fixed->getChars();
|
|
||||||
const auto & in_chars = column_fixed_string->getChars();
|
|
||||||
const auto size = column_fixed_string->size();
|
|
||||||
out_chars.resize_fill(size * n);
|
|
||||||
|
|
||||||
for (const auto i : ext::range(0, size))
|
|
||||||
memcpy(&out_chars[i * n], &in_chars[i * src_n], src_n);
|
|
||||||
|
|
||||||
block.getByPosition(result).column = std::move(column_fixed);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
/// Monotonicity.
|
/// Monotonicity.
|
||||||
|
|
||||||
|
68
src/Functions/globalVariable.cpp
Normal file
68
src/Functions/globalVariable.cpp
Normal file
@ -0,0 +1,68 @@
|
|||||||
|
#include <Functions/IFunctionImpl.h>
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <Functions/FunctionHelpers.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnConst.h>
|
||||||
|
#include <Core/Field.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int BAD_ARGUMENTS;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/** globalVariable('name') - takes constant string argument and returns the value of global variable with that name.
|
||||||
|
* It is intended for compatibility with MySQL.
|
||||||
|
*
|
||||||
|
* Currently it's a stub, no variables are implemented. Feel free to add more variables.
|
||||||
|
*/
|
||||||
|
class FunctionGlobalVariable : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "globalVariable";
|
||||||
|
static FunctionPtr create(const Context &)
|
||||||
|
{
|
||||||
|
return std::make_shared<FunctionGlobalVariable>();
|
||||||
|
}
|
||||||
|
|
||||||
|
String getName() const override
|
||||||
|
{
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override
|
||||||
|
{
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||||
|
{
|
||||||
|
if (!checkColumnConst<ColumnString>(arguments[0].column.get()))
|
||||||
|
throw Exception("Agrument of function " + getName() + " must be constant string", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
|
String variable_name = assert_cast<const ColumnConst &>(*arguments[0].column).getValue<String>();
|
||||||
|
|
||||||
|
throw Exception("There is no global variable with name " + variable_name, ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
}
|
||||||
|
|
||||||
|
void executeImpl(Block & block, const ColumnNumbers &, size_t /*result*/, size_t /*input_rows_count*/) override
|
||||||
|
{
|
||||||
|
String variable_name = assert_cast<const ColumnConst &>(*block.getByPosition(0).column).getValue<String>();
|
||||||
|
|
||||||
|
throw Exception("There is no global variable with name " + variable_name, ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
void registerFunctionGlobalVariable(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionGlobalVariable>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -56,6 +56,7 @@ void registerFunctionTransform(FunctionFactory &);
|
|||||||
void registerFunctionGetMacro(FunctionFactory &);
|
void registerFunctionGetMacro(FunctionFactory &);
|
||||||
void registerFunctionGetScalar(FunctionFactory &);
|
void registerFunctionGetScalar(FunctionFactory &);
|
||||||
void registerFunctionIsConstant(FunctionFactory &);
|
void registerFunctionIsConstant(FunctionFactory &);
|
||||||
|
void registerFunctionGlobalVariable(FunctionFactory &);
|
||||||
|
|
||||||
#if USE_ICU
|
#if USE_ICU
|
||||||
void registerFunctionConvertCharset(FunctionFactory &);
|
void registerFunctionConvertCharset(FunctionFactory &);
|
||||||
@ -112,6 +113,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
|||||||
registerFunctionGetMacro(factory);
|
registerFunctionGetMacro(factory);
|
||||||
registerFunctionGetScalar(factory);
|
registerFunctionGetScalar(factory);
|
||||||
registerFunctionIsConstant(factory);
|
registerFunctionIsConstant(factory);
|
||||||
|
registerFunctionGlobalVariable(factory);
|
||||||
|
|
||||||
#if USE_ICU
|
#if USE_ICU
|
||||||
registerFunctionConvertCharset(factory);
|
registerFunctionConvertCharset(factory);
|
||||||
|
13
src/Functions/toFixedString.cpp
Normal file
13
src/Functions/toFixedString.cpp
Normal file
@ -0,0 +1,13 @@
|
|||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <Functions/toFixedString.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerFunctionFixedString(FunctionFactory & factory)
|
||||||
|
{
|
||||||
|
factory.registerFunction<FunctionToFixedString>();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
110
src/Functions/toFixedString.h
Normal file
110
src/Functions/toFixedString.h
Normal file
@ -0,0 +1,110 @@
|
|||||||
|
#include <Functions/IFunctionImpl.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypeFixedString.h>
|
||||||
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <Columns/ColumnString.h>
|
||||||
|
#include <Columns/ColumnFixedString.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int ILLEGAL_COLUMN;
|
||||||
|
extern const int TOO_LARGE_STRING_SIZE;
|
||||||
|
extern const int NOT_IMPLEMENTED;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/** Conversion to fixed string is implemented only for strings.
|
||||||
|
*/
|
||||||
|
class FunctionToFixedString : public IFunction
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
static constexpr auto name = "toFixedString";
|
||||||
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionToFixedString>(); }
|
||||||
|
static FunctionPtr create() { return std::make_shared<FunctionToFixedString>(); }
|
||||||
|
|
||||||
|
String getName() const override
|
||||||
|
{
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
|
bool isInjective(const Block &) const override { return true; }
|
||||||
|
|
||||||
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||||
|
{
|
||||||
|
if (!isUnsignedInteger(arguments[1].type))
|
||||||
|
throw Exception("Second argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
if (!arguments[1].column)
|
||||||
|
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
if (!isStringOrFixedString(arguments[0].type))
|
||||||
|
throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED);
|
||||||
|
|
||||||
|
const size_t n = arguments[1].column->getUInt(0);
|
||||||
|
return std::make_shared<DataTypeFixedString>(n);
|
||||||
|
}
|
||||||
|
|
||||||
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||||
|
{
|
||||||
|
const auto n = block.getByPosition(arguments[1]).column->getUInt(0);
|
||||||
|
return executeForN(block, arguments, result, n);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void executeForN(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
|
||||||
|
{
|
||||||
|
const auto & column = block.getByPosition(arguments[0]).column;
|
||||||
|
|
||||||
|
if (const auto column_string = checkAndGetColumn<ColumnString>(column.get()))
|
||||||
|
{
|
||||||
|
auto column_fixed = ColumnFixedString::create(n);
|
||||||
|
|
||||||
|
auto & out_chars = column_fixed->getChars();
|
||||||
|
const auto & in_chars = column_string->getChars();
|
||||||
|
const auto & in_offsets = column_string->getOffsets();
|
||||||
|
|
||||||
|
out_chars.resize_fill(in_offsets.size() * n);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < in_offsets.size(); ++i)
|
||||||
|
{
|
||||||
|
const size_t off = i ? in_offsets[i - 1] : 0;
|
||||||
|
const size_t len = in_offsets[i] - off - 1;
|
||||||
|
if (len > n)
|
||||||
|
throw Exception("String too long for type FixedString(" + toString(n) + ")",
|
||||||
|
ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||||
|
memcpy(&out_chars[i * n], &in_chars[off], len);
|
||||||
|
}
|
||||||
|
|
||||||
|
block.getByPosition(result).column = std::move(column_fixed);
|
||||||
|
}
|
||||||
|
else if (const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(column.get()))
|
||||||
|
{
|
||||||
|
const auto src_n = column_fixed_string->getN();
|
||||||
|
if (src_n > n)
|
||||||
|
throw Exception{"String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE};
|
||||||
|
|
||||||
|
auto column_fixed = ColumnFixedString::create(n);
|
||||||
|
|
||||||
|
auto & out_chars = column_fixed->getChars();
|
||||||
|
const auto & in_chars = column_fixed_string->getChars();
|
||||||
|
const auto size = column_fixed_string->size();
|
||||||
|
out_chars.resize_fill(size * n);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < size; ++i)
|
||||||
|
memcpy(&out_chars[i * n], &in_chars[i * src_n], src_n);
|
||||||
|
|
||||||
|
block.getByPosition(result).column = std::move(column_fixed);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -200,6 +200,7 @@ SRCS(
|
|||||||
getMacro.cpp
|
getMacro.cpp
|
||||||
getScalar.cpp
|
getScalar.cpp
|
||||||
getSizeOfEnumType.cpp
|
getSizeOfEnumType.cpp
|
||||||
|
globalVariable.cpp
|
||||||
greatCircleDistance.cpp
|
greatCircleDistance.cpp
|
||||||
greater.cpp
|
greater.cpp
|
||||||
greaterOrEquals.cpp
|
greaterOrEquals.cpp
|
||||||
@ -374,6 +375,7 @@ SRCS(
|
|||||||
toDayOfMonth.cpp
|
toDayOfMonth.cpp
|
||||||
toDayOfWeek.cpp
|
toDayOfWeek.cpp
|
||||||
toDayOfYear.cpp
|
toDayOfYear.cpp
|
||||||
|
toFixedString.cpp
|
||||||
toHour.cpp
|
toHour.cpp
|
||||||
toISOWeek.cpp
|
toISOWeek.cpp
|
||||||
toISOYear.cpp
|
toISOYear.cpp
|
||||||
|
@ -1254,6 +1254,37 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool ParserMySQLGlobalVariable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
|
{
|
||||||
|
if (pos->type != TokenType::DoubleAt)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
++pos;
|
||||||
|
|
||||||
|
if (pos->type != TokenType::BareWord)
|
||||||
|
{
|
||||||
|
expected.add(pos, "variable name");
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
String name(pos->begin, pos->end);
|
||||||
|
++pos;
|
||||||
|
|
||||||
|
auto name_literal = std::make_shared<ASTLiteral>(name);
|
||||||
|
|
||||||
|
auto expr_list_args = std::make_shared<ASTExpressionList>();
|
||||||
|
expr_list_args->children.push_back(std::move(name_literal));
|
||||||
|
|
||||||
|
auto function_node = std::make_shared<ASTFunction>();
|
||||||
|
function_node->name = "globalVariable";
|
||||||
|
function_node->arguments = expr_list_args;
|
||||||
|
function_node->children.push_back(expr_list_args);
|
||||||
|
|
||||||
|
node = function_node;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||||
{
|
{
|
||||||
return ParserSubquery().parse(pos, node, expected)
|
return ParserSubquery().parse(pos, node, expected)
|
||||||
@ -1276,7 +1307,8 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
|||||||
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|
|| ParserQualifiedAsterisk().parse(pos, node, expected)
|
||||||
|| ParserAsterisk().parse(pos, node, expected)
|
|| ParserAsterisk().parse(pos, node, expected)
|
||||||
|| ParserCompoundIdentifier().parse(pos, node, expected)
|
|| ParserCompoundIdentifier().parse(pos, node, expected)
|
||||||
|| ParserSubstitution().parse(pos, node, expected);
|
|| ParserSubstitution().parse(pos, node, expected)
|
||||||
|
|| ParserMySQLGlobalVariable().parse(pos, node, expected);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -301,6 +301,16 @@ protected:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/** MySQL-style global variable: @@var
|
||||||
|
*/
|
||||||
|
class ParserMySQLGlobalVariable : public IParserBase
|
||||||
|
{
|
||||||
|
protected:
|
||||||
|
const char * getName() const override { return "MySQL-style global variable"; }
|
||||||
|
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
/** The expression element is one of: an expression in parentheses, an array, a literal, a function, an identifier, an asterisk.
|
/** The expression element is one of: an expression in parentheses, an array, a literal, a function, an identifier, an asterisk.
|
||||||
*/
|
*/
|
||||||
class ParserExpressionElement : public IParserBase
|
class ParserExpressionElement : public IParserBase
|
||||||
|
@ -305,7 +305,12 @@ Token Lexer::nextTokenImpl()
|
|||||||
return Token(TokenType::ErrorSinglePipeMark, token_begin, pos);
|
return Token(TokenType::ErrorSinglePipeMark, token_begin, pos);
|
||||||
}
|
}
|
||||||
case '@':
|
case '@':
|
||||||
return Token(TokenType::At, token_begin, ++pos);
|
{
|
||||||
|
++pos;
|
||||||
|
if (pos < end && *pos == '@')
|
||||||
|
return Token(TokenType::DoubleAt, token_begin, ++pos);
|
||||||
|
return Token(TokenType::At, token_begin, pos);
|
||||||
|
}
|
||||||
|
|
||||||
default:
|
default:
|
||||||
if (isWordCharASCII(*pos))
|
if (isWordCharASCII(*pos))
|
||||||
|
@ -48,7 +48,8 @@ namespace DB
|
|||||||
M(GreaterOrEquals) \
|
M(GreaterOrEquals) \
|
||||||
M(Concatenation) /** String concatenation operator: || */ \
|
M(Concatenation) /** String concatenation operator: || */ \
|
||||||
\
|
\
|
||||||
M(At) /** @. Used only for specifying user names. */ \
|
M(At) /** @. Used for specifying user names and also for MySQL-style variables. */ \
|
||||||
|
M(DoubleAt) /** @@. Used for MySQL-style global variables. */ \
|
||||||
\
|
\
|
||||||
/** Order is important. EndOfStream goes after all usual tokens, and special error tokens goes after EndOfStream. */ \
|
/** Order is important. EndOfStream goes after all usual tokens, and special error tokens goes after EndOfStream. */ \
|
||||||
\
|
\
|
||||||
|
@ -1,15 +1,20 @@
|
|||||||
#include <Columns/ColumnConst.h>
|
#include <Columns/ColumnConst.h>
|
||||||
|
#include <Columns/ColumnTuple.h>
|
||||||
|
#include <Columns/ColumnsNumber.h>
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
#include <DataTypes/DataTypeNullable.h>
|
#include <DataTypes/DataTypeNullable.h>
|
||||||
|
#include <DataTypes/DataTypeString.h>
|
||||||
|
#include <DataTypes/DataTypeArray.h>
|
||||||
|
#include <DataTypes/DataTypeTuple.h>
|
||||||
#include <DataTypes/FieldToDataType.h>
|
#include <DataTypes/FieldToDataType.h>
|
||||||
#include <Processors/Formats/IRowInputFormat.h>
|
#include <Processors/Formats/IRowInputFormat.h>
|
||||||
#include <Functions/FunctionsConversion.h>
|
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
#include <Interpreters/ExpressionAnalyzer.h>
|
#include <Interpreters/ExpressionAnalyzer.h>
|
||||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||||
#include <Interpreters/SyntaxAnalyzer.h>
|
#include <Interpreters/SyntaxAnalyzer.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/convertFieldToType.h>
|
#include <Interpreters/convertFieldToType.h>
|
||||||
|
#include <Interpreters/ExpressionActions.h>
|
||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
|
@ -0,0 +1 @@
|
|||||||
|
SELECT @@test; -- { serverError 36 }
|
Loading…
Reference in New Issue
Block a user