ClickHouse/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp

398 lines
14 KiB
C++
Raw Normal View History

#include <IO/ReadHelpers.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
#include <Interpreters/convertFieldToType.h>
#include <Parsers/TokenIterator.h>
#include <Parsers/ExpressionListParsers.h>
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
#include <Formats/FormatFactory.h>
#include <Common/FieldVisitors.h>
#include <Core/Block.h>
#include <Common/typeid_cast.h>
#include <common/find_symbols.h>
2019-09-11 19:55:28 +00:00
#include <Parsers/ASTLiteral.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int CANNOT_PARSE_NUMBER;
extern const int CANNOT_PARSE_DATE;
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
extern const int CANNOT_PARSE_DATE;
extern const int SYNTAX_ERROR;
extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE;
2019-09-11 19:55:28 +00:00
extern const int SUPPORT_IS_DISABLED;
}
ValuesBlockInputFormat::ValuesBlockInputFormat(ReadBuffer & in_, const Block & header_, const RowInputFormatParams & params_,
const Context & context_, const FormatSettings & format_settings_)
: IInputFormat(header_, buf), buf(in_), params(params_), context(std::make_unique<Context>(context_)),
format_settings(format_settings_), num_columns(header_.columns()),
2019-09-13 17:40:48 +00:00
parser_type_for_column(num_columns, ParserType::Streaming),
attempts_to_deduce_template(num_columns), attempts_to_deduce_template_cached(num_columns),
rows_parsed_using_template(num_columns), templates(num_columns)
{
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
skipBOMIfExists(buf);
}
Chunk ValuesBlockInputFormat::generate()
{
const Block & header = getPort().getHeader();
2019-05-16 02:05:44 +00:00
MutableColumns columns = header.cloneEmptyColumns();
for (size_t rows_in_block = 0, batch = 0; rows_in_block < params.max_block_size; ++rows_in_block, ++batch)
{
if (params.rows_portion_size && batch == params.rows_portion_size)
2019-09-02 12:57:22 +00:00
{
batch = 0;
if (!checkTimeLimit(params, total_stopwatch) || isCancelled())
2019-09-02 12:57:22 +00:00
break;
}
try
{
skipWhitespaceIfAny(buf);
if (buf.eof() || *buf.position() == ';')
2019-05-16 02:05:44 +00:00
break;
assertChar('(', buf);
2019-05-16 02:05:44 +00:00
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
{
skipWhitespaceIfAny(buf);
PeekableReadBufferCheckpoint checkpoint{buf};
2019-05-16 02:05:44 +00:00
/// Parse value using fast streaming parser for literals and slow SQL parser for expressions.
2019-09-13 17:40:48 +00:00
/// If there is SQL expression in some row, template of this expression will be deduced,
/// so it makes possible to parse the following rows much faster
/// if expressions in the following rows have the same structure
if (parser_type_for_column[column_idx] == ParserType::Streaming)
tryReadValue(*columns[column_idx], column_idx);
else if (parser_type_for_column[column_idx] == ParserType::BatchTemplate)
tryParseExpressionUsingTemplate(columns[column_idx], column_idx);
else /// if (parser_type_for_column[column_idx] == ParserType::SingleExpressionEvaluation)
parseExpression(*columns[column_idx], column_idx);
}
2019-05-12 03:15:08 +00:00
skipWhitespaceIfAny(buf);
if (!buf.eof() && *buf.position() == ',')
++buf.position();
++total_rows;
}
catch (Exception & e)
{
if (isParseError(e.code()))
e.addMessage(" at row " + std::to_string(total_rows));
throw;
}
}
2019-05-16 02:05:44 +00:00
/// Evaluate expressions, which were parsed using templates, if any
for (size_t i = 0; i < columns.size(); ++i)
{
2019-09-11 19:55:28 +00:00
if (!templates[i] || !templates[i]->rowsCount())
continue;
if (columns[i]->empty())
2019-09-11 19:55:28 +00:00
columns[i] = std::move(*templates[i]->evaluateAll()).mutate();
else
{
2019-09-11 19:55:28 +00:00
ColumnPtr evaluated = templates[i]->evaluateAll();
columns[i]->insertRangeFrom(*evaluated, 0, evaluated->size());
}
}
if (columns.empty() || columns[0]->empty())
{
readSuffix();
return {};
}
size_t rows_in_block = columns[0]->size();
return Chunk{std::move(columns), rows_in_block};
}
2019-09-13 17:40:48 +00:00
void ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx)
{
2019-09-13 17:40:48 +00:00
if (!templates[column_idx])
throw Exception("bug", ErrorCodes::LOGICAL_ERROR);
2019-09-11 19:55:28 +00:00
/// Try to parse expression using template if one was successfully deduced while parsing the first row
if (templates[column_idx]->parseExpression(buf, format_settings))
{
2019-09-11 19:55:28 +00:00
++rows_parsed_using_template[column_idx];
return;
}
2019-09-11 19:55:28 +00:00
/// Expression in the current row is not match template deduced on the first row.
/// Evaluate expressions, which were parsed using this template.
if (column->empty())
column = std::move(*templates[column_idx]->evaluateAll()).mutate();
else
{
ColumnPtr evaluated = templates[column_idx]->evaluateAll();
column->insertRangeFrom(*evaluated, 0, evaluated->size());
}
/// Do not use this template anymore
templates[column_idx].reset();
buf.rollbackToCheckpoint();
/// It will deduce new template or fallback to slow SQL parser
2019-09-13 17:40:48 +00:00
parseExpression(*column, column_idx);
}
2019-09-13 17:40:48 +00:00
void ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx)
2019-05-16 02:05:44 +00:00
{
2019-09-13 17:40:48 +00:00
bool rollback_on_exception = false;
2019-05-16 02:05:44 +00:00
try
{
2019-09-13 17:40:48 +00:00
const Block & header = getPort().getHeader();
header.getByPosition(column_idx).type->deserializeAsTextQuoted(column, buf, format_settings);
rollback_on_exception = true;
2019-05-16 02:05:44 +00:00
2019-09-13 17:40:48 +00:00
skipWhitespaceIfAny(buf);
assertDelimiterAfterValue(column_idx);
2019-05-16 02:05:44 +00:00
}
catch (const Exception & e)
{
2019-09-13 17:40:48 +00:00
if (!isParseError(e.code()))
2019-05-16 02:05:44 +00:00
throw;
2019-09-13 17:40:48 +00:00
if (rollback_on_exception)
column.popBack(1);
/// Switch to SQL parser and don't try to use streaming parser for complex expressions
/// Note: Throwing exceptions for each expression may be very slow because of stacktraces
buf.rollbackToCheckpoint();
parseExpression(column, column_idx);
2019-05-16 02:05:44 +00:00
}
}
void
2019-09-13 17:40:48 +00:00
ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
{
2019-09-13 17:40:48 +00:00
parser_type_for_column[column_idx] = ParserType::SingleExpressionEvaluation;
2019-09-11 19:55:28 +00:00
const Block & header = getPort().getHeader();
const IDataType & type = *header.getByPosition(column_idx).type;
/// We need continuous memory containing the expression to use Lexer
skipToNextRow(0, 1);
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
Expected expected;
Tokens tokens(buf.position(), buf.buffer().end());
IParser::Pos token_iterator(tokens);
ASTPtr ast;
bool parsed = parser.parse(token_iterator, ast, expected);
2019-09-06 19:01:44 +00:00
/// Consider delimiter after value (',' or ')') as part of expression
if (column_idx + 1 != num_columns)
parsed &= token_iterator->type == TokenType::Comma;
else
parsed &= token_iterator->type == TokenType::ClosingRoundBracket;
if (!parsed)
throw Exception("Cannot parse expression of type " + type.getName() + " here: "
+ String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())),
ErrorCodes::SYNTAX_ERROR);
++token_iterator;
2019-09-13 17:40:48 +00:00
if (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
/// Check if we can use fast streaming parser instead if using templates
bool rollback_on_exception = false;
bool ok = false;
try
{
header.getByPosition(column_idx).type->deserializeAsTextQuoted(column, buf, format_settings);
rollback_on_exception = true;
skipWhitespaceIfAny(buf);
if (checkDelimiterAfterValue(column_idx))
ok = true;
}
catch (const Exception & e)
{
if (!isParseError(e.code()))
throw;
}
if (ok)
{
parser_type_for_column[column_idx] = ParserType::Streaming;
return;
}
else if (rollback_on_exception)
column.popBack(1);
}
/// Try to deduce template of expression and use it to parse the following rows
if (shouldDeduceNewTemplate(column_idx))
{
2019-05-16 02:05:44 +00:00
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);
try
{
2019-09-13 17:40:48 +00:00
bool found_in_cache = false;
const auto & result_type = header.getByPosition(column_idx).type;
const char * delimiter = (column_idx + 1 == num_columns) ? ")" : ",";
auto structure = templates_cache.getFromCacheOrConstruct(result_type, TokenIterator(tokens), token_iterator,
ast, *context, &found_in_cache, delimiter);
templates[column_idx].emplace(structure);
if (found_in_cache)
++attempts_to_deduce_template_cached[column_idx];
else
++attempts_to_deduce_template[column_idx];
buf.rollbackToCheckpoint();
2019-09-11 19:55:28 +00:00
templates[column_idx]->parseExpression(buf, format_settings);
++rows_parsed_using_template[column_idx];
2019-09-13 17:40:48 +00:00
parser_type_for_column[column_idx] = ParserType::BatchTemplate;
2019-05-16 02:05:44 +00:00
return;
}
2019-05-16 02:05:44 +00:00
catch (...)
{
if (!format_settings.values.interpret_expressions)
throw;
/// Continue parsing without template
templates[column_idx].reset();
}
}
2019-05-16 02:05:44 +00:00
2019-09-13 17:40:48 +00:00
if (!format_settings.values.interpret_expressions)
throw Exception("Interpreting expressions is disabled", ErrorCodes::SUPPORT_IS_DISABLED);
/// Try to evaluate single expression if other parsers don't work
buf.position() = const_cast<char *>(token_iterator->begin);
2019-09-04 19:42:01 +00:00
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(ast, *context);
Field value = convertFieldToType(value_raw.first, type, value_raw.second.get());
/// Check that we are indeed allowed to insert a NULL.
if (value.isNull() && !type.isNullable())
{
buf.rollbackToCheckpoint();
throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value)
+ ", that is out of range of type " + type.getName()
+ ", at: " +
String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())),
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE};
}
2019-05-16 02:05:44 +00:00
column.insert(value);
}
/// Can be used in fileSegmentationEngine for parallel parsing of Values
bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_size, int balance)
{
skipWhitespaceIfAny(buf);
if (buf.eof() || *buf.position() == ';')
return false;
bool quoted = false;
size_t chunk_begin_buf_count = buf.count();
while (!buf.eof() && (balance || buf.count() - chunk_begin_buf_count < min_chunk_size))
{
buf.position() = find_first_symbols<'\\', '\'', ')', '('>(buf.position(), buf.buffer().end());
if (buf.position() == buf.buffer().end())
continue;
if (*buf.position() == '\\')
{
++buf.position();
if (!buf.eof())
++buf.position();
}
else if (*buf.position() == '\'')
{
quoted ^= true;
++buf.position();
}
else if (*buf.position() == ')')
{
++buf.position();
if (!quoted)
--balance;
}
else if (*buf.position() == '(')
{
++buf.position();
if (!quoted)
++balance;
}
}
if (!buf.eof() && *buf.position() == ',')
++buf.position();
return true;
}
void ValuesBlockInputFormat::assertDelimiterAfterValue(size_t column_idx)
2019-05-16 02:05:44 +00:00
{
skipWhitespaceIfAny(buf);
2019-05-16 02:05:44 +00:00
if (column_idx + 1 != num_columns)
assertChar(',', buf);
else
assertChar(')', buf);
}
2019-09-04 19:42:01 +00:00
bool ValuesBlockInputFormat::checkDelimiterAfterValue(size_t column_idx)
{
skipWhitespaceIfAny(buf);
if (column_idx + 1 != num_columns)
return checkChar(',', buf);
else
return checkChar(')', buf);
}
bool ValuesBlockInputFormat::shouldDeduceNewTemplate(size_t column_idx)
{
2019-09-11 19:55:28 +00:00
if (!format_settings.values.deduce_templates_of_expressions)
return false;
2019-09-04 19:42:01 +00:00
/// TODO better heuristic
2019-09-13 17:40:48 +00:00
/// Using template from cache is approx 2x faster, than evaluating single expression
/// Construction of new template is approx 1.5x slower, than evaluating single expression
float attempts_weighted = 1.5 * attempts_to_deduce_template[column_idx] + 0.5 * attempts_to_deduce_template_cached[column_idx];
constexpr size_t max_attempts = 100;
if (attempts_weighted < max_attempts)
return true;
2019-09-13 17:40:48 +00:00
if (rows_parsed_using_template[column_idx] / attempts_weighted > 1)
{
/// Try again
2019-09-04 19:42:01 +00:00
attempts_to_deduce_template[column_idx] = 0;
2019-09-13 17:40:48 +00:00
attempts_to_deduce_template_cached[column_idx] = 0;
rows_parsed_using_template[column_idx] = 0;
return true;
}
return false;
}
void registerInputFormatProcessorValues(FormatFactory & factory)
{
factory.registerInputFormatProcessor("Values", [](
ReadBuffer & buf,
const Block & header,
const Context & context,
const RowInputFormatParams & params,
const FormatSettings & settings)
{
return std::make_shared<ValuesBlockInputFormat>(buf, header, params, context, settings);
});
}
}