mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
fixes after review
This commit is contained in:
parent
605046761e
commit
80e5976d10
@ -2186,6 +2186,7 @@ public:
|
|||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 2; }
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
|
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
|
|
||||||
|
@ -258,6 +258,12 @@ public:
|
|||||||
/// You pass data types with empty DataTypeFunction for lambda arguments.
|
/// You pass data types with empty DataTypeFunction for lambda arguments.
|
||||||
/// This function will replace it with DataTypeFunction containing actual types.
|
/// This function will replace it with DataTypeFunction containing actual types.
|
||||||
virtual void getLambdaArgumentTypes(DataTypes & arguments) const = 0;
|
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<IFunctionBuilder>;
|
using FunctionBuilderPtr = std::shared_ptr<IFunctionBuilder>;
|
||||||
@ -281,6 +287,9 @@ public:
|
|||||||
getLambdaArgumentTypesImpl(arguments);
|
getLambdaArgumentTypesImpl(arguments);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
|
||||||
|
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {}; }
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
|
/// 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
|
virtual DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
|
||||||
@ -501,7 +510,11 @@ public:
|
|||||||
bool isVariadic() const override { return function->isVariadic(); }
|
bool isVariadic() const override { return function->isVariadic(); }
|
||||||
size_t getNumberOfArguments() const override { return function->getNumberOfArguments(); }
|
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:
|
protected:
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { return function->getReturnTypeImpl(arguments); }
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { return function->getReturnTypeImpl(arguments); }
|
||||||
|
@ -31,6 +31,7 @@ public:
|
|||||||
size_t getNumberOfArguments() const override { return 1; }
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; }
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
|
@ -36,6 +36,13 @@ public:
|
|||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
bool isVariadic() const override { return true; }
|
bool isVariadic() const override { return true; }
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
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
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
|
@ -895,6 +895,7 @@ public:
|
|||||||
size_t getNumberOfArguments() const override { return 3; }
|
size_t getNumberOfArguments() const override { return 3; }
|
||||||
|
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
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.
|
/// 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
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
|
@ -34,6 +34,7 @@ public:
|
|||||||
size_t getNumberOfArguments() const override { return 2; }
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; }
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
|
@ -30,6 +30,7 @@ public:
|
|||||||
size_t getNumberOfArguments() const override { return 1; }
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; }
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
{
|
{
|
||||||
|
@ -29,6 +29,7 @@ public:
|
|||||||
size_t getNumberOfArguments() const override { return 1; }
|
size_t getNumberOfArguments() const override { return 1; }
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; }
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes &) const override
|
||||||
{
|
{
|
||||||
|
@ -40,6 +40,13 @@ public:
|
|||||||
bool isVariadic() const override { return true; }
|
bool isVariadic() const override { return true; }
|
||||||
size_t getNumberOfArguments() const override { return 0; }
|
size_t getNumberOfArguments() const override { return 0; }
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
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
|
DataTypePtr getReturnTypeImpl(const DataTypes & args) const override
|
||||||
{
|
{
|
||||||
|
@ -80,6 +80,7 @@ protected:
|
|||||||
|
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
|
||||||
|
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -25,6 +25,7 @@ public:
|
|||||||
}
|
}
|
||||||
|
|
||||||
bool useDefaultImplementationForNulls() const override { return false; }
|
bool useDefaultImplementationForNulls() const override { return false; }
|
||||||
|
ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; }
|
||||||
|
|
||||||
/// Get the name of the function.
|
/// Get the name of the function.
|
||||||
String getName() const override
|
String getName() const override
|
||||||
|
@ -16,6 +16,7 @@
|
|||||||
#include <Processors/Formats/Impl/ConstantExpressionTemplate.h>
|
#include <Processors/Formats/Impl/ConstantExpressionTemplate.h>
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
#include <Parsers/ExpressionElementParsers.h>
|
||||||
#include <Interpreters/convertFieldToType.h>
|
#include <Interpreters/convertFieldToType.h>
|
||||||
|
#include <boost/functional/hash.hpp>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -66,10 +67,10 @@ public:
|
|||||||
return;
|
return;
|
||||||
if (auto function = ast->as<ASTFunction>())
|
if (auto function = ast->as<ASTFunction>())
|
||||||
visit(*function, force_nullable);
|
visit(*function, force_nullable);
|
||||||
else if (ast->as<ASTLiteral>())
|
else if (ast->as<ASTIdentifier>())
|
||||||
throw DB::Exception("Identifier in constant expression", ErrorCodes::SYNTAX_ERROR);
|
throw DB::Exception("Identifier in constant expression", ErrorCodes::SYNTAX_ERROR);
|
||||||
else
|
else
|
||||||
visitChildren(ast, {}, std::vector<char>(ast->children.size(), force_nullable));
|
throw DB::Exception("Syntax error in constant expression", ErrorCodes::SYNTAX_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
private:
|
private:
|
||||||
@ -85,26 +86,16 @@ private:
|
|||||||
if (function.name == "lambda")
|
if (function.name == "lambda")
|
||||||
return;
|
return;
|
||||||
|
|
||||||
/// Do not replace literals which must be constant
|
|
||||||
ColumnNumbers dont_visit_children;
|
|
||||||
FunctionBuilderPtr builder = FunctionFactory::instance().get(function.name, context);
|
FunctionBuilderPtr builder = FunctionFactory::instance().get(function.name, context);
|
||||||
|
/// Do not replace literals which must be constant
|
||||||
if (auto default_builder = dynamic_cast<DefaultFunctionBuilder*>(builder.get()))
|
ColumnNumbers dont_visit_children = builder->getArgumentsThatAreAlwaysConstant();
|
||||||
dont_visit_children = default_builder->getArgumentsThatAreAlwaysConstant();
|
|
||||||
else if (dynamic_cast<FunctionBuilderCast*>(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)
|
|
||||||
|
|
||||||
/// Allow nullable arguments if function never returns NULL
|
/// Allow nullable arguments if function never returns NULL
|
||||||
bool return_not_null = function.name == "isNull" || function.name == "isNotNull" ||
|
ColumnNumbers can_always_be_nullable = builder->getArgumentsThatDontImplyNullableReturnType(function.arguments->children.size());
|
||||||
function.name == "ifNull" || function.name == "assumeNotNull" ||
|
|
||||||
function.name == "coalesce";
|
|
||||||
|
|
||||||
std::vector<char> force_nullable_arguments(function.arguments->children.size(), force_nullable || return_not_null);
|
std::vector<char> force_nullable_arguments(function.arguments->children.size(), force_nullable);
|
||||||
|
for (auto & idx : can_always_be_nullable)
|
||||||
/// coalesce may return NULL if the last argument is NULL
|
if (idx < force_nullable_arguments.size())
|
||||||
if (!force_nullable && function.name == "coalesce")
|
force_nullable_arguments[idx] = true;
|
||||||
force_nullable_arguments.back() = false;
|
|
||||||
|
|
||||||
visitChildren(function.arguments, dont_visit_children, force_nullable_arguments);
|
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
|
/// Allows distinguish expression in the last column in Values format
|
||||||
hash_state.update(salt);
|
hash_state.update(salt);
|
||||||
|
|
||||||
IAST::Hash res;
|
IAST::Hash res128;
|
||||||
hash_state.get128(res.first, res.second);
|
hash_state.get128(res128.first, res128.second);
|
||||||
return res.first ^ res.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;
|
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<TemplateStructure>(visitor.replaced_literals, expression_begin, expression_end, expression, *result_column_type, context);
|
|
||||||
columns = structure->literals.cloneEmptyColumns();
|
|
||||||
}
|
|
||||||
|
|
||||||
bool ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const FormatSettings & settings)
|
bool ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const FormatSettings & settings)
|
||||||
{
|
{
|
||||||
size_t cur_column = 0;
|
size_t cur_column = 0;
|
||||||
@ -451,7 +433,9 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co
|
|||||||
istr.position() += 4;
|
istr.position() += 4;
|
||||||
else
|
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() == '-';
|
bool negative = *istr.position() == '-';
|
||||||
if (negative || *istr.position() == '+')
|
if (negative || *istr.position() == '+')
|
||||||
++istr.position();
|
++istr.position();
|
||||||
|
@ -48,6 +48,7 @@ public:
|
|||||||
public:
|
public:
|
||||||
explicit Cache(size_t max_size_ = 4096) : max_size(max_size_) {}
|
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,
|
TemplateStructurePtr getFromCacheOrConstruct(const DataTypePtr & result_column_type,
|
||||||
TokenIterator expression_begin,
|
TokenIterator expression_begin,
|
||||||
TokenIterator expression_end,
|
TokenIterator expression_end,
|
||||||
@ -60,10 +61,6 @@ public:
|
|||||||
explicit ConstantExpressionTemplate(const TemplateStructurePtr & structure_)
|
explicit ConstantExpressionTemplate(const TemplateStructurePtr & structure_)
|
||||||
: structure(structure_), columns(structure->literals.cloneEmptyColumns()) {}
|
: 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)
|
/// Read expression from istr, assert it has the same structure and the same types of literals (template matches)
|
||||||
/// and parse literals into temporary columns
|
/// and parse literals into temporary columns
|
||||||
bool parseExpression(ReadBuffer & istr, const FormatSettings & settings);
|
bool parseExpression(ReadBuffer & istr, const FormatSettings & settings);
|
||||||
|
@ -56,6 +56,8 @@ Chunk ValuesBlockInputFormat::generate()
|
|||||||
if (buf.eof() || *buf.position() == ';')
|
if (buf.eof() || *buf.position() == ';')
|
||||||
break;
|
break;
|
||||||
readRow(columns);
|
readRow(columns);
|
||||||
|
if (params.callback)
|
||||||
|
params.callback();
|
||||||
}
|
}
|
||||||
catch (Exception & e)
|
catch (Exception & e)
|
||||||
{
|
{
|
||||||
@ -119,8 +121,6 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns)
|
|||||||
|
|
||||||
void ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx)
|
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
|
/// Try to parse expression using template if one was successfully deduced while parsing the first row
|
||||||
if (templates[column_idx]->parseExpression(buf, format_settings))
|
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)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
if (!isParseError(e.code()))
|
if (!isParseError(e.code()) && e.code() != ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED)
|
||||||
throw;
|
throw;
|
||||||
if (rollback_on_exception)
|
if (rollback_on_exception)
|
||||||
column.popBack(1);
|
column.popBack(1);
|
||||||
@ -173,8 +173,6 @@ void ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx)
|
|||||||
void
|
void
|
||||||
ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
|
ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
|
||||||
{
|
{
|
||||||
parser_type_for_column[column_idx] = ParserType::SingleExpressionEvaluation;
|
|
||||||
|
|
||||||
const Block & header = getPort().getHeader();
|
const Block & header = getPort().getHeader();
|
||||||
const IDataType & type = *header.getByPosition(column_idx).type;
|
const IDataType & type = *header.getByPosition(column_idx).type;
|
||||||
|
|
||||||
@ -202,7 +200,7 @@ ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
|
|||||||
ErrorCodes::SYNTAX_ERROR);
|
ErrorCodes::SYNTAX_ERROR);
|
||||||
++token_iterator;
|
++token_iterator;
|
||||||
|
|
||||||
if (dynamic_cast<const ASTLiteral *>(ast.get()))
|
if (parser_type_for_column[column_idx] != ParserType::Streaming && dynamic_cast<const ASTLiteral *>(ast.get()))
|
||||||
{
|
{
|
||||||
/// It's possible that streaming parsing has failed on some row (e.g. because of '+' sign before integer),
|
/// 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
|
/// but it still can parse the following rows
|
||||||
@ -231,12 +229,15 @@ ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
|
|||||||
column.popBack(1);
|
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
|
/// Try to deduce template of expression and use it to parse the following rows
|
||||||
if (shouldDeduceNewTemplate(column_idx))
|
if (shouldDeduceNewTemplate(column_idx))
|
||||||
{
|
{
|
||||||
if (templates[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",
|
throw DB::Exception("Template for column " + std::to_string(column_idx) + " already exists and it was not evaluated yet",
|
||||||
ErrorCodes::LOGICAL_ERROR);
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
std::exception_ptr exception;
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
bool found_in_cache = false;
|
bool found_in_cache = false;
|
||||||
@ -251,18 +252,30 @@ ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
|
|||||||
++attempts_to_deduce_template[column_idx];
|
++attempts_to_deduce_template[column_idx];
|
||||||
|
|
||||||
buf.rollbackToCheckpoint();
|
buf.rollbackToCheckpoint();
|
||||||
templates[column_idx]->parseExpression(buf, format_settings);
|
if (templates[column_idx]->parseExpression(buf, format_settings))
|
||||||
++rows_parsed_using_template[column_idx];
|
{
|
||||||
parser_type_for_column[column_idx] = ParserType::BatchTemplate;
|
++rows_parsed_using_template[column_idx];
|
||||||
return;
|
parser_type_for_column[column_idx] = ParserType::BatchTemplate;
|
||||||
|
return;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
if (!format_settings.values.interpret_expressions)
|
exception = std::current_exception();
|
||||||
throw;
|
|
||||||
/// Continue parsing without template
|
|
||||||
templates[column_idx].reset();
|
|
||||||
}
|
}
|
||||||
|
if (!format_settings.values.interpret_expressions)
|
||||||
|
{
|
||||||
|
if (exception)
|
||||||
|
std::rethrow_exception(exception);
|
||||||
|
else
|
||||||
|
{
|
||||||
|
buf.rollbackToCheckpoint();
|
||||||
|
size_t len = const_cast<char *>(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)
|
if (!format_settings.values.interpret_expressions)
|
||||||
|
@ -2,7 +2,15 @@
|
|||||||
1970-01-03 world 7 2 inf []
|
1970-01-03 world 7 2 inf []
|
||||||
1970-01-04 test 6 -9223372036854775807 3.14 [5,4]
|
1970-01-04 test 6 -9223372036854775807 3.14 [5,4]
|
||||||
1970-01-05 expressions 15 -9223372036854775807 1 [0,9,8,7,6]
|
1970-01-05 expressions 15 -9223372036854775807 1 [0,9,8,7,6]
|
||||||
1970-01-02 hello 6
|
1970-01-02 hello 6 [NULL,NULL]
|
||||||
1970-01-03 warld 5
|
1970-01-03 warld 5 [-1,NULL,1]
|
||||||
1970-01-04 test 4
|
1970-01-04 test 4 [NULL,1,3.14]
|
||||||
1970-01-05 \N \N
|
1970-01-05 \N \N [1,2,3]
|
||||||
|
1
|
||||||
|
2
|
||||||
|
3
|
||||||
|
7
|
||||||
|
11
|
||||||
|
16
|
||||||
|
24
|
||||||
|
32
|
||||||
|
@ -2,7 +2,8 @@ DROP TABLE IF EXISTS values_template;
|
|||||||
DROP TABLE IF EXISTS values_template_nullable;
|
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 (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_interpret_expressions = 0;
|
||||||
SET input_format_values_deduce_templates_of_expressions = 1;
|
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
|
-- _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 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 ORDER BY d;
|
||||||
SELECT * FROM values_template_nullable 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;
|
||||||
DROP TABLE values_template_nullable;
|
DROP TABLE values_template_nullable;
|
||||||
|
DROP TABLE values_template_fallback;
|
||||||
|
@ -215,7 +215,7 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), (
|
|||||||
This feature is experimental, disabled by default.
|
This feature is experimental, disabled by default.
|
||||||
|
|
||||||
## input_format_values_accurate_types_of_literals {#settings-input_format_values_accurate_types_of_literals}
|
## 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
|
```sql
|
||||||
(..., abs(0), ...), -- UInt64 literal
|
(..., abs(0), ...), -- UInt64 literal
|
||||||
(..., abs(3.141592654), ...), -- Float64 literal
|
(..., abs(3.141592654), ...), -- Float64 literal
|
||||||
|
Loading…
Reference in New Issue
Block a user