2019-05-07 22:39:18 +00:00
|
|
|
#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 <Formats/ValuesBlockInputStream.h>
|
|
|
|
#include <Formats/FormatFactory.h>
|
|
|
|
#include <Formats/BlockInputStreamFromRowInputStream.h>
|
|
|
|
#include <Common/FieldVisitors.h>
|
|
|
|
#include <Core/Block.h>
|
|
|
|
#include <Common/typeid_cast.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-05-08 01:41:05 +00:00
|
|
|
extern const int CANNOT_CREATE_EXPRESSION_TEMPLATE;
|
|
|
|
extern const int CANNOT_PARSE_EXPRESSION_USING_TEMPLATE;
|
|
|
|
extern const int CANNOT_EVALUATE_EXPRESSION_TEMPLATE;
|
2019-05-07 22:39:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
ValuesBlockInputStream::ValuesBlockInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_,
|
2019-05-23 03:07:49 +00:00
|
|
|
const FormatSettings & format_settings, UInt64 max_block_size_, UInt64 rows_portion_size_)
|
|
|
|
: istr(istr_), header(header_), context(std::make_unique<Context>(context_)), format_settings(format_settings),
|
|
|
|
max_block_size(max_block_size_), rows_portion_size(rows_portion_size_), num_columns(header.columns()),
|
|
|
|
attempts_to_generate_template(num_columns), rows_parsed_using_template(num_columns)
|
2019-05-07 22:39:18 +00:00
|
|
|
{
|
2019-05-08 01:41:05 +00:00
|
|
|
templates.resize(header.columns());
|
2019-05-07 22:39:18 +00:00
|
|
|
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
|
|
|
|
skipBOMIfExists(istr);
|
|
|
|
}
|
|
|
|
|
2019-05-16 02:05:44 +00:00
|
|
|
Block ValuesBlockInputStream::readImpl()
|
2019-05-07 22:39:18 +00:00
|
|
|
{
|
2019-05-16 02:05:44 +00:00
|
|
|
MutableColumns columns = header.cloneEmptyColumns();
|
2019-05-07 22:39:18 +00:00
|
|
|
|
2019-05-16 02:05:44 +00:00
|
|
|
for (size_t rows_in_block = 0; rows_in_block < max_block_size; ++rows_in_block)
|
2019-05-07 22:39:18 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
skipWhitespaceIfAny(istr);
|
2019-05-16 02:05:44 +00:00
|
|
|
if (istr.eof() || *istr.position() == ';')
|
|
|
|
break;
|
|
|
|
assertChar('(', istr);
|
2019-05-07 22:39:18 +00:00
|
|
|
|
2019-05-16 02:05:44 +00:00
|
|
|
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
|
2019-05-07 22:39:18 +00:00
|
|
|
{
|
2019-05-16 02:05:44 +00:00
|
|
|
skipWhitespaceIfAny(istr);
|
2019-05-23 03:07:49 +00:00
|
|
|
PeekableReadBufferCheckpoint checkpoint{istr};
|
|
|
|
|
|
|
|
bool parse_separate_value = !parseExpressionUsingTemplate(columns[column_idx], column_idx);
|
2019-05-12 03:15:08 +00:00
|
|
|
|
2019-05-16 02:05:44 +00:00
|
|
|
/// Parse value using fast streaming parser for literals and slow SQL parser for expressions.
|
|
|
|
/// If there is SQL expression in the first row, template of this expression will be generated,
|
|
|
|
/// so it makes possible to parse next rows much faster if expressions in next rows have the same structure
|
|
|
|
if (parse_separate_value)
|
2019-05-23 03:07:49 +00:00
|
|
|
readValue(*columns[column_idx], column_idx, shouldGenerateNewTemplate(column_idx));
|
2019-05-07 22:39:18 +00:00
|
|
|
}
|
2019-05-12 03:15:08 +00:00
|
|
|
|
2019-05-16 02:05:44 +00:00
|
|
|
skipWhitespaceIfAny(istr);
|
|
|
|
if (!istr.eof() && *istr.position() == ',')
|
|
|
|
++istr.position();
|
2019-05-07 22:39:18 +00:00
|
|
|
|
2019-05-08 01:41:05 +00:00
|
|
|
++total_rows;
|
2019-05-07 22:39:18 +00:00
|
|
|
}
|
|
|
|
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
|
2019-05-08 01:41:05 +00:00
|
|
|
for (size_t i = 0; i < columns.size(); ++i)
|
|
|
|
{
|
2019-05-23 03:07:49 +00:00
|
|
|
if (!templates[i] || !templates[i].value().rowsCount())
|
|
|
|
continue;
|
|
|
|
if (columns[i]->empty())
|
2019-05-08 01:41:05 +00:00
|
|
|
columns[i] = std::move(*templates[i].value().evaluateAll()).mutate();
|
2019-05-23 03:07:49 +00:00
|
|
|
else
|
|
|
|
{
|
|
|
|
ColumnPtr evaluated = templates[i].value().evaluateAll();
|
|
|
|
columns[i]->insertRangeFrom(*evaluated, 0, evaluated->size());
|
2019-05-08 01:41:05 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-05-07 22:39:18 +00:00
|
|
|
if (columns.empty() || columns[0]->empty())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
return header.cloneWithColumns(std::move(columns));
|
|
|
|
}
|
|
|
|
|
2019-05-23 03:07:49 +00:00
|
|
|
bool ValuesBlockInputStream::parseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx)
|
|
|
|
{
|
|
|
|
if (templates[column_idx])
|
|
|
|
{
|
|
|
|
/// Try to parse expression using template if one was successfully generated while parsing the first row
|
|
|
|
try
|
|
|
|
{
|
|
|
|
templates[column_idx].value().parseExpression(istr, format_settings);
|
|
|
|
assertDelimAfterValue(column_idx);
|
|
|
|
++rows_parsed_using_template[column_idx];
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
catch (DB::Exception & e)
|
|
|
|
{
|
|
|
|
if (e.code() != ErrorCodes::CANNOT_PARSE_EXPRESSION_USING_TEMPLATE)
|
|
|
|
throw;
|
|
|
|
/// Expression in the current row is not match generated on the first row template.
|
|
|
|
/// Evaluate expressions, which were parsed using this template.
|
|
|
|
if (column->empty())
|
|
|
|
column = std::move(*templates[column_idx].value().evaluateAll()).mutate();
|
|
|
|
else
|
|
|
|
{
|
|
|
|
ColumnPtr evaluated = templates[column_idx].value().evaluateAll();
|
|
|
|
column->insertRangeFrom(*evaluated, 0, evaluated->size());
|
|
|
|
}
|
|
|
|
/// Do not use the template anymore and fallback to slow SQL parser
|
|
|
|
templates[column_idx].reset();
|
|
|
|
++attempts_to_generate_template[column_idx];
|
|
|
|
istr.rollbackToCheckpoint();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2019-05-16 02:05:44 +00:00
|
|
|
void ValuesBlockInputStream::readValue(IColumn & column, size_t column_idx, bool generate_template)
|
|
|
|
{
|
|
|
|
bool rollback_on_exception = false;
|
|
|
|
try
|
|
|
|
{
|
|
|
|
header.getByPosition(column_idx).type->deserializeAsTextQuoted(column, istr, format_settings);
|
|
|
|
rollback_on_exception = true;
|
|
|
|
|
|
|
|
skipWhitespaceIfAny(istr);
|
|
|
|
|
|
|
|
assertDelimAfterValue(column_idx);
|
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
2019-05-23 01:30:00 +00:00
|
|
|
if (!format_settings.values.interpret_expressions && !(format_settings.values.deduce_templates_of_expressions && generate_template))
|
2019-05-16 02:05:44 +00:00
|
|
|
throw;
|
|
|
|
|
|
|
|
/** The normal streaming parser could not parse the value.
|
|
|
|
* Let's try to parse it with a SQL parser as a constant expression.
|
|
|
|
* This is an exceptional case.
|
|
|
|
*/
|
|
|
|
if (e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_QUOTED_STRING
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_NUMBER
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_DATE
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_PARSE_DATETIME
|
|
|
|
|| e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT)
|
|
|
|
{
|
|
|
|
if (rollback_on_exception)
|
|
|
|
column.popBack(1);
|
|
|
|
|
|
|
|
istr.rollbackToCheckpoint();
|
|
|
|
parseExpression(column, column_idx, generate_template);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void
|
|
|
|
ValuesBlockInputStream::parseExpression(IColumn & column, size_t column_idx, bool generate_template)
|
2019-05-08 01:41:05 +00:00
|
|
|
{
|
|
|
|
const IDataType & type = *header.getByPosition(column_idx).type;
|
|
|
|
|
|
|
|
Expected expected;
|
|
|
|
|
2019-05-12 03:15:08 +00:00
|
|
|
// TODO make tokenizer to work with buffers, not only with continuous memory
|
|
|
|
Tokens tokens(istr.position(), istr.buffer().end());
|
2019-05-08 01:41:05 +00:00
|
|
|
TokenIterator token_iterator(tokens);
|
|
|
|
|
|
|
|
ASTPtr ast;
|
|
|
|
if (!parser.parse(token_iterator, ast, expected))
|
2019-05-12 03:15:08 +00:00
|
|
|
{
|
|
|
|
istr.rollbackToCheckpoint();
|
2019-05-08 01:41:05 +00:00
|
|
|
throw Exception("Cannot parse expression of type " + type.getName() + " here: "
|
2019-05-12 03:15:08 +00:00
|
|
|
+ String(istr.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - istr.position())),
|
2019-05-08 01:41:05 +00:00
|
|
|
ErrorCodes::SYNTAX_ERROR);
|
2019-05-12 03:15:08 +00:00
|
|
|
}
|
2019-05-08 01:41:05 +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.
|
2019-05-12 03:15:08 +00:00
|
|
|
if (value.isNull() && !type.isNullable())
|
2019-05-08 01:41:05 +00:00
|
|
|
{
|
2019-05-12 03:15:08 +00:00
|
|
|
istr.rollbackToCheckpoint();
|
|
|
|
throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value)
|
|
|
|
+ ", that is out of range of type " + type.getName()
|
|
|
|
+ ", at: " +
|
|
|
|
String(istr.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - istr.position())),
|
|
|
|
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE};
|
2019-05-08 01:41:05 +00:00
|
|
|
}
|
|
|
|
|
2019-05-12 03:15:08 +00:00
|
|
|
istr.position() = const_cast<char *>(token_iterator->begin);
|
|
|
|
|
2019-05-23 01:30:00 +00:00
|
|
|
if (format_settings.values.deduce_templates_of_expressions && generate_template)
|
2019-05-08 01:41:05 +00:00
|
|
|
{
|
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);
|
2019-05-08 01:41:05 +00:00
|
|
|
try
|
|
|
|
{
|
2019-05-22 03:29:32 +00:00
|
|
|
templates[column_idx] = ConstantExpressionTemplate(type, TokenIterator(tokens), token_iterator, ast, *context);
|
2019-05-12 03:15:08 +00:00
|
|
|
istr.rollbackToCheckpoint();
|
2019-05-08 01:41:05 +00:00
|
|
|
templates[column_idx].value().parseExpression(istr, format_settings);
|
2019-05-16 02:05:44 +00:00
|
|
|
assertDelimAfterValue(column_idx);
|
|
|
|
return;
|
2019-05-08 01:41:05 +00:00
|
|
|
}
|
2019-05-16 02:05:44 +00:00
|
|
|
catch (...)
|
2019-05-08 01:41:05 +00:00
|
|
|
{
|
2019-05-23 01:30:00 +00:00
|
|
|
if (!format_settings.values.interpret_expressions)
|
|
|
|
throw;
|
2019-05-08 01:41:05 +00:00
|
|
|
/// Continue parsing without template
|
|
|
|
templates[column_idx].reset();
|
|
|
|
istr.position() = const_cast<char *>(token_iterator->begin);
|
|
|
|
}
|
|
|
|
}
|
2019-05-16 02:05:44 +00:00
|
|
|
|
|
|
|
assertDelimAfterValue(column_idx);
|
|
|
|
column.insert(value);
|
|
|
|
}
|
|
|
|
|
|
|
|
void ValuesBlockInputStream::assertDelimAfterValue(size_t column_idx)
|
|
|
|
{
|
|
|
|
skipWhitespaceIfAny(istr);
|
|
|
|
|
|
|
|
if (column_idx + 1 != num_columns)
|
|
|
|
assertChar(',', istr);
|
2019-05-08 01:41:05 +00:00
|
|
|
else
|
2019-05-16 02:05:44 +00:00
|
|
|
assertChar(')', istr);
|
2019-05-08 01:41:05 +00:00
|
|
|
}
|
|
|
|
|
2019-05-23 03:07:49 +00:00
|
|
|
bool ValuesBlockInputStream::shouldGenerateNewTemplate(size_t column_idx)
|
|
|
|
{
|
|
|
|
// TODO better heuristic
|
|
|
|
constexpr size_t max_attempts = 3;
|
|
|
|
constexpr size_t rows_per_attempt = 10;
|
|
|
|
if (attempts_to_generate_template[column_idx] < max_attempts)
|
|
|
|
return true;
|
|
|
|
if (rows_parsed_using_template[column_idx] / attempts_to_generate_template[column_idx] < rows_per_attempt)
|
|
|
|
{
|
|
|
|
/// Try again
|
|
|
|
attempts_to_generate_template[column_idx] = 0;
|
|
|
|
rows_parsed_using_template[column_idx] = 0;
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2019-05-07 22:39:18 +00:00
|
|
|
|
|
|
|
void registerInputFormatValues(FormatFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerInputFormat("Values", [](
|
|
|
|
ReadBuffer & buf,
|
|
|
|
const Block & sample,
|
|
|
|
const Context & context,
|
|
|
|
UInt64 max_block_size,
|
|
|
|
const FormatSettings & settings)
|
|
|
|
{
|
|
|
|
return std::make_shared<ValuesBlockInputStream>(buf, sample, context, settings, max_block_size);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|