diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e396ae9c868..5e0bc081526 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -398,6 +398,7 @@ private: { TokenType::GreaterOrEquals, Replxx::Color::INTENSE }, { TokenType::Concatenation, Replxx::Color::INTENSE }, { TokenType::At, Replxx::Color::INTENSE }, + { TokenType::DoubleAt, Replxx::Color::MAGENTA }, { TokenType::EndOfStream, Replxx::Color::DEFAULT }, diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index bbde6e04069..ef1e78baa16 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -11,6 +11,8 @@ FunctionOverloadResolverImplPtr CastOverloadResolver::create(const Context & con return createImpl(context.getSettingsRef().cast_keep_nullable); } +void registerFunctionFixedString(FunctionFactory & factory); + void registerFunctionsConversion(FunctionFactory & factory) { factory.registerFunction(); @@ -33,7 +35,8 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); + + registerFunctionFixedString(factory); factory.registerFunction(); factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index b23cac8c456..6d046e4ef7e 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -41,6 +41,7 @@ #include #include #include +#include namespace DB @@ -58,7 +59,6 @@ namespace ErrorCodes extern const int CANNOT_PARSE_DATETIME; extern const int CANNOT_PARSE_TEXT; extern const int CANNOT_PARSE_UUID; - extern const int TOO_LARGE_STRING_SIZE; extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int LOGICAL_ERROR; 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(); } - static FunctionPtr create() { return std::make_shared(); } - - 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(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(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(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. diff --git a/src/Functions/globalVariable.cpp b/src/Functions/globalVariable.cpp new file mode 100644 index 00000000000..1443422daee --- /dev/null +++ b/src/Functions/globalVariable.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include +#include +#include + + +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(); + } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!checkColumnConst(arguments[0].column.get())) + throw Exception("Agrument of function " + getName() + " must be constant string", ErrorCodes::BAD_ARGUMENTS); + + String variable_name = assert_cast(*arguments[0].column).getValue(); + + 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(*block.getByPosition(0).column).getValue(); + + throw Exception("There is no global variable with name " + variable_name, ErrorCodes::BAD_ARGUMENTS); + } +}; + + +void registerFunctionGlobalVariable(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 3e6099ee25a..0025a856c4b 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -56,6 +56,7 @@ void registerFunctionTransform(FunctionFactory &); void registerFunctionGetMacro(FunctionFactory &); void registerFunctionGetScalar(FunctionFactory &); void registerFunctionIsConstant(FunctionFactory &); +void registerFunctionGlobalVariable(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -112,6 +113,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionGetMacro(factory); registerFunctionGetScalar(factory); registerFunctionIsConstant(factory); + registerFunctionGlobalVariable(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/toFixedString.cpp b/src/Functions/toFixedString.cpp new file mode 100644 index 00000000000..cfc357a055a --- /dev/null +++ b/src/Functions/toFixedString.cpp @@ -0,0 +1,13 @@ +#include +#include + + +namespace DB +{ + +void registerFunctionFixedString(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/toFixedString.h b/src/Functions/toFixedString.h new file mode 100644 index 00000000000..f55e242b570 --- /dev/null +++ b/src/Functions/toFixedString.h @@ -0,0 +1,110 @@ +#include +#include +#include +#include +#include +#include +#include + + +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(); } + static FunctionPtr create() { return std::make_shared(); } + + 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(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(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(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); + } +}; + +} + diff --git a/src/Functions/ya.make b/src/Functions/ya.make index ea66828c942..80ffbbede87 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -200,6 +200,7 @@ SRCS( getMacro.cpp getScalar.cpp getSizeOfEnumType.cpp + globalVariable.cpp greatCircleDistance.cpp greater.cpp greaterOrEquals.cpp @@ -374,6 +375,7 @@ SRCS( toDayOfMonth.cpp toDayOfWeek.cpp toDayOfYear.cpp + toFixedString.cpp toHour.cpp toISOWeek.cpp toISOYear.cpp diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 0466c6f96f9..ba33a5823d1 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.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(name); + + auto expr_list_args = std::make_shared(); + expr_list_args->children.push_back(std::move(name_literal)); + + auto function_node = std::make_shared(); + 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) { return ParserSubquery().parse(pos, node, expected) @@ -1276,7 +1307,8 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp || ParserQualifiedAsterisk().parse(pos, node, expected) || ParserAsterisk().parse(pos, node, expected) || ParserCompoundIdentifier().parse(pos, node, expected) - || ParserSubstitution().parse(pos, node, expected); + || ParserSubstitution().parse(pos, node, expected) + || ParserMySQLGlobalVariable().parse(pos, node, expected); } diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index f5cd705bb83..238d582f0b0 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -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. */ class ParserExpressionElement : public IParserBase diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 7efc8663416..edce730bf90 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -305,7 +305,12 @@ Token Lexer::nextTokenImpl() return Token(TokenType::ErrorSinglePipeMark, token_begin, pos); } 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: if (isWordCharASCII(*pos)) diff --git a/src/Parsers/Lexer.h b/src/Parsers/Lexer.h index b4b2bc96157..dc1c9824b6b 100644 --- a/src/Parsers/Lexer.h +++ b/src/Parsers/Lexer.h @@ -48,7 +48,8 @@ namespace DB M(GreaterOrEquals) \ 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. */ \ \ diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index f121b85d255..1ae094f4ded 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -1,15 +1,20 @@ #include +#include +#include #include #include +#include +#include +#include #include #include -#include #include #include #include #include #include #include +#include #include #include #include diff --git a/tests/queries/0_stateless/01337_mysql_global_variables.reference b/tests/queries/0_stateless/01337_mysql_global_variables.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01337_mysql_global_variables.sql b/tests/queries/0_stateless/01337_mysql_global_variables.sql new file mode 100644 index 00000000000..a4f2c3c4148 --- /dev/null +++ b/tests/queries/0_stateless/01337_mysql_global_variables.sql @@ -0,0 +1 @@ +SELECT @@test; -- { serverError 36 }