diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 29aee42d001..c16f439c3fa 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -2186,6 +2186,7 @@ public: size_t getNumberOfArguments() const override { return 2; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } protected: diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index 0a180ee03eb..ae0b380bd97 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -258,6 +258,12 @@ public: /// You pass data types with empty DataTypeFunction for lambda arguments. /// This function will replace it with DataTypeFunction containing actual types. virtual void getLambdaArgumentTypes(DataTypes & arguments) const = 0; + + /// Returns indexes of arguments, that must be ColumnConst + virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const = 0; + /// Returns indexes if arguments, that can be Nullable without making result of function Nullable + /// (for functions like isNull(x)) + virtual ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const = 0; }; using FunctionBuilderPtr = std::shared_ptr; @@ -281,6 +287,9 @@ public: getLambdaArgumentTypesImpl(arguments); } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {}; } + protected: /// Get the result type by argument type. If the function does not apply to these arguments, throw an exception. virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const @@ -501,7 +510,11 @@ public: bool isVariadic() const override { return function->isVariadic(); } size_t getNumberOfArguments() const override { return function->getNumberOfArguments(); } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return function->getArgumentsThatAreAlwaysConstant(); } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return function->getArgumentsThatAreAlwaysConstant(); } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override + { + return function->getArgumentsThatDontImplyNullableReturnType(number_of_arguments); + } protected: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { return function->getReturnTypeImpl(arguments); } diff --git a/dbms/src/Functions/assumeNotNull.cpp b/dbms/src/Functions/assumeNotNull.cpp index 5439b4eb7a3..dc3f2b9bb5d 100644 --- a/dbms/src/Functions/assumeNotNull.cpp +++ b/dbms/src/Functions/assumeNotNull.cpp @@ -31,6 +31,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/dbms/src/Functions/coalesce.cpp b/dbms/src/Functions/coalesce.cpp index 947ecd0e199..72732f1f11c 100644 --- a/dbms/src/Functions/coalesce.cpp +++ b/dbms/src/Functions/coalesce.cpp @@ -36,6 +36,13 @@ public: bool useDefaultImplementationForNulls() const override { return false; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override + { + ColumnNumbers args; + for (size_t i = 0; i + 1 < number_of_arguments; ++i) + args.push_back(i); + return args; + } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/dbms/src/Functions/if.cpp b/dbms/src/Functions/if.cpp index 99740b26914..f0534a13d66 100644 --- a/dbms/src/Functions/if.cpp +++ b/dbms/src/Functions/if.cpp @@ -895,6 +895,7 @@ public: size_t getNumberOfArguments() const override { return 3; } bool useDefaultImplementationForNulls() const override { return false; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } /// Get result types by argument types. If the function does not apply to these arguments, throw an exception. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override diff --git a/dbms/src/Functions/ifNull.cpp b/dbms/src/Functions/ifNull.cpp index 2c552d86ffe..895c935bdd3 100644 --- a/dbms/src/Functions/ifNull.cpp +++ b/dbms/src/Functions/ifNull.cpp @@ -34,6 +34,7 @@ public: size_t getNumberOfArguments() const override { return 2; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { diff --git a/dbms/src/Functions/isNotNull.cpp b/dbms/src/Functions/isNotNull.cpp index 5762dbdbfcc..60a7f4f9239 100644 --- a/dbms/src/Functions/isNotNull.cpp +++ b/dbms/src/Functions/isNotNull.cpp @@ -30,6 +30,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes &) const override { diff --git a/dbms/src/Functions/isNull.cpp b/dbms/src/Functions/isNull.cpp index cc68753ea27..441aded5c04 100644 --- a/dbms/src/Functions/isNull.cpp +++ b/dbms/src/Functions/isNull.cpp @@ -29,6 +29,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } DataTypePtr getReturnTypeImpl(const DataTypes &) const override { diff --git a/dbms/src/Functions/multiIf.cpp b/dbms/src/Functions/multiIf.cpp index bf64fd9af4c..d943c110fa4 100644 --- a/dbms/src/Functions/multiIf.cpp +++ b/dbms/src/Functions/multiIf.cpp @@ -40,6 +40,13 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForNulls() const override { return false; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override + { + ColumnNumbers args; + for (size_t i = 0; i + 1 < number_of_arguments; i += 2) + args.push_back(i); + return args; + } DataTypePtr getReturnTypeImpl(const DataTypes & args) const override { diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp index 84ab32c3643..5adcb62133c 100644 --- a/dbms/src/Functions/toTypeName.cpp +++ b/dbms/src/Functions/toTypeName.cpp @@ -80,6 +80,7 @@ protected: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } }; diff --git a/dbms/src/Functions/visibleWidth.cpp b/dbms/src/Functions/visibleWidth.cpp index 8d229e878a8..6e81ec2ffd5 100644 --- a/dbms/src/Functions/visibleWidth.cpp +++ b/dbms/src/Functions/visibleWidth.cpp @@ -25,6 +25,7 @@ public: } bool useDefaultImplementationForNulls() const override { return false; } + ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } /// Get the name of the function. String getName() const override diff --git a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index d69526c0aff..2ce3cbe4078 100644 --- a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -66,10 +67,10 @@ public: return; if (auto function = ast->as()) visit(*function, force_nullable); - else if (ast->as()) + else if (ast->as()) throw DB::Exception("Identifier in constant expression", ErrorCodes::SYNTAX_ERROR); else - visitChildren(ast, {}, std::vector(ast->children.size(), force_nullable)); + throw DB::Exception("Syntax error in constant expression", ErrorCodes::SYNTAX_ERROR); } private: @@ -85,26 +86,16 @@ private: if (function.name == "lambda") return; - /// Do not replace literals which must be constant - ColumnNumbers dont_visit_children; FunctionBuilderPtr builder = FunctionFactory::instance().get(function.name, context); - - if (auto default_builder = dynamic_cast(builder.get())) - dont_visit_children = default_builder->getArgumentsThatAreAlwaysConstant(); - else if (dynamic_cast(builder.get())) - dont_visit_children.push_back(1); - /// FIXME suppose there is no other functions, which require constant arguments (it's true, until the new one is added) - + /// Do not replace literals which must be constant + ColumnNumbers dont_visit_children = builder->getArgumentsThatAreAlwaysConstant(); /// Allow nullable arguments if function never returns NULL - bool return_not_null = function.name == "isNull" || function.name == "isNotNull" || - function.name == "ifNull" || function.name == "assumeNotNull" || - function.name == "coalesce"; + ColumnNumbers can_always_be_nullable = builder->getArgumentsThatDontImplyNullableReturnType(function.arguments->children.size()); - std::vector force_nullable_arguments(function.arguments->children.size(), force_nullable || return_not_null); - - /// coalesce may return NULL if the last argument is NULL - if (!force_nullable && function.name == "coalesce") - force_nullable_arguments.back() = false; + std::vector force_nullable_arguments(function.arguments->children.size(), force_nullable); + for (auto & idx : can_always_be_nullable) + if (idx < force_nullable_arguments.size()) + force_nullable_arguments[idx] = true; visitChildren(function.arguments, dont_visit_children, force_nullable_arguments); } @@ -281,9 +272,12 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP /// Allows distinguish expression in the last column in Values format hash_state.update(salt); - IAST::Hash res; - hash_state.get128(res.first, res.second); - return res.first ^ res.second; + IAST::Hash res128; + hash_state.get128(res128.first, res128.second); + size_t res = 0; + boost::hash_combine(res, res128.first); + boost::hash_combine(res, res128.second); + return res; } @@ -324,18 +318,6 @@ ConstantExpressionTemplate::Cache::getFromCacheOrConstruct(const DataTypePtr & r return res; } -ConstantExpressionTemplate::ConstantExpressionTemplate(const DataTypePtr & result_column_type, - TokenIterator expression_begin, TokenIterator expression_end, - const ASTPtr & expression_, const Context & context) -{ - ASTPtr expression = expression_->clone(); - ReplaceLiteralsVisitor visitor(context); - visitor.visit(expression, result_column_type->isNullable()); - - structure = std::make_shared(visitor.replaced_literals, expression_begin, expression_end, expression, *result_column_type, context); - columns = structure->literals.cloneEmptyColumns(); -} - bool ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const FormatSettings & settings) { size_t cur_column = 0; @@ -451,7 +433,9 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co istr.position() += 4; else { - /// ParserNumber::parse(...) is slow because of using ASTPtr, Expected and Tokens, which is't needed here + /// ParserNumber::parse(...) is about 20x slower than strtod(...) + /// because of using ASTPtr, Expected and Tokens, which are not needed here. + /// Parse numeric literal in the same way, as ParserNumber does, but use strtod and strtoull directly. bool negative = *istr.position() == '-'; if (negative || *istr.position() == '+') ++istr.position(); diff --git a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index 7222fb62cab..4f4dac849b2 100644 --- a/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/dbms/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -48,6 +48,7 @@ public: public: explicit Cache(size_t max_size_ = 4096) : max_size(max_size_) {} + /// Deduce template of expression of type result_column_type and add it to cache (or use template from cache) TemplateStructurePtr getFromCacheOrConstruct(const DataTypePtr & result_column_type, TokenIterator expression_begin, TokenIterator expression_end, @@ -60,10 +61,6 @@ public: explicit ConstantExpressionTemplate(const TemplateStructurePtr & structure_) : structure(structure_), columns(structure->literals.cloneEmptyColumns()) {} - /// Deduce template of expression of type result_column_type - ConstantExpressionTemplate(const DataTypePtr & result_column_type_, TokenIterator expression_begin_, TokenIterator expression_end_, - const ASTPtr & expression_, const Context & context_); - /// Read expression from istr, assert it has the same structure and the same types of literals (template matches) /// and parse literals into temporary columns bool parseExpression(ReadBuffer & istr, const FormatSettings & settings); diff --git a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index d8b89de42ba..8fda4294e3d 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -56,6 +56,8 @@ Chunk ValuesBlockInputFormat::generate() if (buf.eof() || *buf.position() == ';') break; readRow(columns); + if (params.callback) + params.callback(); } catch (Exception & e) { @@ -119,8 +121,6 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns) void ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx) { - if (!templates[column_idx]) - throw Exception("bug", ErrorCodes::LOGICAL_ERROR); /// Try to parse expression using template if one was successfully deduced while parsing the first row if (templates[column_idx]->parseExpression(buf, format_settings)) { @@ -158,7 +158,7 @@ void ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) } catch (const Exception & e) { - if (!isParseError(e.code())) + if (!isParseError(e.code()) && e.code() != ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED) throw; if (rollback_on_exception) column.popBack(1); @@ -173,8 +173,6 @@ void ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) void ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx) { - parser_type_for_column[column_idx] = ParserType::SingleExpressionEvaluation; - const Block & header = getPort().getHeader(); const IDataType & type = *header.getByPosition(column_idx).type; @@ -202,7 +200,7 @@ ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx) ErrorCodes::SYNTAX_ERROR); ++token_iterator; - if (dynamic_cast(ast.get())) + if (parser_type_for_column[column_idx] != ParserType::Streaming && dynamic_cast(ast.get())) { /// It's possible that streaming parsing has failed on some row (e.g. because of '+' sign before integer), /// but it still can parse the following rows @@ -231,12 +229,15 @@ ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx) column.popBack(1); } + parser_type_for_column[column_idx] = ParserType::SingleExpressionEvaluation; + /// Try to deduce template of expression and use it to parse the following rows if (shouldDeduceNewTemplate(column_idx)) { if (templates[column_idx]) throw DB::Exception("Template for column " + std::to_string(column_idx) + " already exists and it was not evaluated yet", ErrorCodes::LOGICAL_ERROR); + std::exception_ptr exception; try { bool found_in_cache = false; @@ -251,18 +252,30 @@ ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx) ++attempts_to_deduce_template[column_idx]; buf.rollbackToCheckpoint(); - templates[column_idx]->parseExpression(buf, format_settings); - ++rows_parsed_using_template[column_idx]; - parser_type_for_column[column_idx] = ParserType::BatchTemplate; - return; + if (templates[column_idx]->parseExpression(buf, format_settings)) + { + ++rows_parsed_using_template[column_idx]; + parser_type_for_column[column_idx] = ParserType::BatchTemplate; + return; + } } catch (...) { - if (!format_settings.values.interpret_expressions) - throw; - /// Continue parsing without template - templates[column_idx].reset(); + exception = std::current_exception(); } + if (!format_settings.values.interpret_expressions) + { + if (exception) + std::rethrow_exception(exception); + else + { + buf.rollbackToCheckpoint(); + size_t len = const_cast(token_iterator->begin) - buf.position(); + throw Exception("Cannot deduce template of expression: " + std::string(buf.position(), len), ErrorCodes::SYNTAX_ERROR); + } + } + /// Continue parsing without template + templates[column_idx].reset(); } if (!format_settings.values.interpret_expressions) diff --git a/dbms/tests/queries/0_stateless/00948_values_interpreter_template.reference b/dbms/tests/queries/0_stateless/00948_values_interpreter_template.reference index 7bb92e22620..7559603fed6 100644 --- a/dbms/tests/queries/0_stateless/00948_values_interpreter_template.reference +++ b/dbms/tests/queries/0_stateless/00948_values_interpreter_template.reference @@ -2,7 +2,15 @@ 1970-01-03 world 7 2 inf [] 1970-01-04 test 6 -9223372036854775807 3.14 [5,4] 1970-01-05 expressions 15 -9223372036854775807 1 [0,9,8,7,6] -1970-01-02 hello 6 -1970-01-03 warld 5 -1970-01-04 test 4 -1970-01-05 \N \N +1970-01-02 hello 6 [NULL,NULL] +1970-01-03 warld 5 [-1,NULL,1] +1970-01-04 test 4 [NULL,1,3.14] +1970-01-05 \N \N [1,2,3] +1 +2 +3 +7 +11 +16 +24 +32 diff --git a/dbms/tests/queries/0_stateless/00948_values_interpreter_template.sql b/dbms/tests/queries/0_stateless/00948_values_interpreter_template.sql index 9a6506a8319..812acfbcda6 100644 --- a/dbms/tests/queries/0_stateless/00948_values_interpreter_template.sql +++ b/dbms/tests/queries/0_stateless/00948_values_interpreter_template.sql @@ -2,7 +2,8 @@ DROP TABLE IF EXISTS values_template; DROP TABLE IF EXISTS values_template_nullable; CREATE TABLE values_template (d Date, s String, u UInt8, i Int64, f Float64, a Array(UInt8)) ENGINE = Memory; -CREATE TABLE values_template_nullable (d Date, s Nullable(String), u Nullable(UInt8)) ENGINE = Memory; +CREATE TABLE values_template_nullable (d Date, s Nullable(String), u Nullable(UInt8), a Array(Nullable(Float32))) ENGINE = Memory; +CREATE TABLE values_template_fallback (n UInt8) ENGINE = Memory; SET input_format_values_interpret_expressions = 0; SET input_format_values_deduce_templates_of_expressions = 1; @@ -13,11 +14,21 @@ SET input_format_values_deduce_templates_of_expressions = 1; -- _NUM_5: Float64 -> Int64 INSERT INTO values_template VALUES ((1), lower(replaceAll('Hella', 'a', 'o')), 1 + 2 + 3, round(-4 * 5.0), nan / CAST('42', 'Int8'), reverse([1, 2, 3])), ((2), lower(replaceAll('Warld', 'a', 'o')), -4 + 5 + 6, round(18446744073709551615 * 1e-19), 1.0 / CAST('0', 'Int8'), reverse([])), ((3), lower(replaceAll('Test', 'a', 'o')), 3 + 2 + 1, round(9223372036854775807 * -1), 6.28 / CAST('2', 'Int8'), reverse([4, 5])), ((4), lower(replaceAll('Expressians', 'a', 'o')), 6 + 5 + 4, round(1 * -9223372036854775807), 127.0 / CAST('127', 'Int8'), reverse([6, 7, 8, 9, 0])); +INSERT INTO values_template_nullable VALUES ((1), lower(replaceAll('Hella', 'a', 'o')), 1 + 2 + 3, arraySort(x -> assumeNotNull(x), [null, NULL])), ((2), lower(replaceAll('Warld', 'b', 'o')), 4 - 5 + 6, arraySort(x -> assumeNotNull(x), [+1, -1, Null])), ((3), lower(replaceAll('Test', 'c', 'o')), 3 + 2 - 1, arraySort(x -> assumeNotNull(x), [1, nUlL, 3.14])), ((4), lower(replaceAll(null, 'c', 'o')), 6 + 5 - null, arraySort(x -> assumeNotNull(x), [3, 2, 1])); -INSERT INTO values_template_nullable VALUES ((1), lower(replaceAll('Hella', 'a', 'o')), 1 + 2 + 3), ((2), lower(replaceAll('Warld', 'b', 'o')), 4 - 5 + 6), ((3), lower(replaceAll('Test', 'c', 'o')), 3 + 2 - 1), ((4), lower(replaceAll(null, 'c', 'o')), 6 + 5 - null); +INSERT INTO values_template_fallback VALUES (1 + x); -- { clientError 62 } +INSERT INTO values_template_fallback VALUES (abs(functionThatDoesNotExists(42))); -- { clientError 46 } +INSERT INTO values_template_fallback VALUES ([1]); -- { clientError 43 } +INSERT INTO values_template_fallback VALUES (CAST(1, 'UInt8')), (CAST('2', 'UInt8')); +SET input_format_values_accurate_types_of_literals = 0; +SET input_format_values_interpret_expressions = 1; +INSERT INTO values_template_fallback VALUES (1 + 2), (3 + +04), (5 + 6); +INSERT INTO values_template_fallback VALUES (+020), (+030), (+040); SELECT * FROM values_template ORDER BY d; SELECT * FROM values_template_nullable ORDER BY d; +SELECT * FROM values_template_fallback ORDER BY n; DROP TABLE values_template; DROP TABLE values_template_nullable; +DROP TABLE values_template_fallback; diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7af6fb81dd3..3f94d8e45c5 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -215,7 +215,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( This feature is experimental, disabled by default. ## input_format_values_accurate_types_of_literals {#settings-input_format_values_accurate_types_of_literals} -This setting is used unly when `input_format_values_deduce_templates_of_expressions = 1`. It can happen, that expressions for some column have the same structure, but contain numeric literals of different types, e.g +This setting is used only when `input_format_values_deduce_templates_of_expressions = 1`. It can happen, that expressions for some column have the same structure, but contain numeric literals of different types, e.g ```sql (..., abs(0), ...), -- UInt64 literal (..., abs(3.141592654), ...), -- Float64 literal