fixes after review

This commit is contained in:
Alexander Tokmakov 2019-10-02 20:51:00 +03:00
parent 605046761e
commit 80e5976d10
17 changed files with 109 additions and 61 deletions

View File

@ -2186,6 +2186,7 @@ public:
size_t getNumberOfArguments() const override { return 2; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
protected:

View File

@ -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<IFunctionBuilder>;
@ -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); }

View File

@ -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
{

View File

@ -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
{

View File

@ -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

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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
{

View File

@ -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}; }
};

View File

@ -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

View File

@ -16,6 +16,7 @@
#include <Processors/Formats/Impl/ConstantExpressionTemplate.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Interpreters/convertFieldToType.h>
#include <boost/functional/hash.hpp>
namespace DB
@ -66,10 +67,10 @@ public:
return;
if (auto function = ast->as<ASTFunction>())
visit(*function, force_nullable);
else if (ast->as<ASTLiteral>())
else if (ast->as<ASTIdentifier>())
throw DB::Exception("Identifier in constant expression", ErrorCodes::SYNTAX_ERROR);
else
visitChildren(ast, {}, std::vector<char>(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<DefaultFunctionBuilder*>(builder.get()))
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)
/// 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<char> 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<char> 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<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)
{
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();

View File

@ -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);

View File

@ -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<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),
/// 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<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)

View File

@ -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

View File

@ -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;

View File

@ -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