fix build

This commit is contained in:
Alexander Tokmakov 2019-09-02 15:57:22 +03:00
parent 3b6a038604
commit 8adf13047b
4 changed files with 21 additions and 9 deletions

View File

@ -3,7 +3,7 @@
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/FieldToDataType.h> #include <DataTypes/FieldToDataType.h>
#include <Formats/BlockInputStreamFromRowInputStream.h> #include <Processors/Formats/IRowInputFormat.h>
#include <Functions/FunctionsConversion.h> #include <Functions/FunctionsConversion.h>
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <Interpreters/ExpressionAnalyzer.h> #include <Interpreters/ExpressionAnalyzer.h>
@ -198,7 +198,7 @@ void ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const Format
type_info = WhichDataType(dynamic_cast<const DataTypeNullable &>(type).getNestedType()); type_info = WhichDataType(dynamic_cast<const DataTypeNullable &>(type).getNestedType());
Tokens tokens_number(istr.position(), istr.buffer().end()); Tokens tokens_number(istr.position(), istr.buffer().end());
TokenIterator iterator(tokens_number); IParser::Pos iterator(tokens_number);
Expected expected; Expected expected;
ASTPtr ast; ASTPtr ast;
if (nullable && parser_null.parse(iterator, ast, expected)) if (nullable && parser_null.parse(iterator, ast, expected))

View File

@ -6,7 +6,7 @@
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
#include <Formats/ValuesBlockInputStream.h> #include <Formats/ValuesBlockInputStream.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <Formats/BlockInputStreamFromRowInputStream.h> #include <Processors/Formats/IRowInputFormat.h>
#include <Common/FieldVisitors.h> #include <Common/FieldVisitors.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
@ -33,8 +33,8 @@ namespace ErrorCodes
ValuesBlockInputStream::ValuesBlockInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, ValuesBlockInputStream::ValuesBlockInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_,
const FormatSettings & format_settings, UInt64 max_block_size_, UInt64 rows_portion_size_) 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), : 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()), 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) attempts_to_generate_template(num_columns), rows_parsed_using_template(num_columns)
{ {
@ -48,8 +48,14 @@ Block ValuesBlockInputStream::readImpl()
{ {
MutableColumns columns = header.cloneEmptyColumns(); MutableColumns columns = header.cloneEmptyColumns();
for (size_t rows_in_block = 0; rows_in_block < max_block_size; ++rows_in_block) for (size_t rows_in_block = 0, batch = 0; rows_in_block < max_block_size; ++rows_in_block, ++batch)
{ {
if (rows_portion_size && batch == rows_portion_size)
{
batch = 0;
if (!checkTimeLimit() || isCancelled())
break;
}
try try
{ {
skipWhitespaceIfAny(istr); skipWhitespaceIfAny(istr);
@ -187,7 +193,7 @@ ValuesBlockInputStream::parseExpression(IColumn & column, size_t column_idx, boo
// TODO make tokenizer to work with buffers, not only with continuous memory // TODO make tokenizer to work with buffers, not only with continuous memory
Tokens tokens(istr.position(), istr.buffer().end()); Tokens tokens(istr.position(), istr.buffer().end());
TokenIterator token_iterator(tokens); IParser::Pos token_iterator(tokens);
ASTPtr ast; ASTPtr ast;
if (!parser.parse(token_iterator, ast, expected)) if (!parser.parse(token_iterator, ast, expected))
@ -276,9 +282,11 @@ void registerInputFormatValues(FormatFactory & factory)
const Block & sample, const Block & sample,
const Context & context, const Context & context,
UInt64 max_block_size, UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback,
const FormatSettings & settings) const FormatSettings & settings)
{ {
return std::make_shared<ValuesBlockInputStream>(buf, sample, context, settings, max_block_size); return std::make_shared<ValuesBlockInputStream>(buf, sample, context, settings, max_block_size, rows_portion_size);
}); });
} }

View File

@ -6,6 +6,7 @@
#include <Formats/ConstantExpressionTemplate.h> #include <Formats/ConstantExpressionTemplate.h>
#include <IO/PeekableReadBuffer.h> #include <IO/PeekableReadBuffer.h>
#include <Parsers/ExpressionListParsers.h>
namespace DB namespace DB
{ {
@ -23,7 +24,7 @@ public:
* If interpret_expressions is true, it will, in addition, try to use SQL parser and interpreter * If interpret_expressions is true, it will, in addition, try to use SQL parser and interpreter
* in case when streaming parser could not parse field (this is very slow). * in case when streaming parser could not parse field (this is very slow).
*/ */
ValuesBlockInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, const FormatSettings & format_settings, UInt64 max_block_size_); ValuesBlockInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, const FormatSettings & format_settings_, UInt64 max_block_size_, UInt64 rows_portion_size_);
String getName() const override { return "ValuesBlockOutputStream"; } String getName() const override { return "ValuesBlockOutputStream"; }
Block getHeader() const override { return header; } Block getHeader() const override { return header; }
@ -50,6 +51,7 @@ private:
std::unique_ptr<Context> context; /// pimpl std::unique_ptr<Context> context; /// pimpl
const FormatSettings format_settings; const FormatSettings format_settings;
UInt64 max_block_size; UInt64 max_block_size;
UInt64 rows_portion_size;
size_t num_columns; size_t num_columns;
size_t total_rows = 0; size_t total_rows = 0;

View File

@ -36,6 +36,8 @@ struct RowInputFormatParams
OverflowMode timeout_overflow_mode = OverflowMode::THROW; OverflowMode timeout_overflow_mode = OverflowMode::THROW;
}; };
bool isParseError(int code);
///Row oriented input format: reads data row by row. ///Row oriented input format: reads data row by row.
class IRowInputFormat : public IInputFormat class IRowInputFormat : public IInputFormat
{ {