mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
fix build
This commit is contained in:
parent
3b6a038604
commit
8adf13047b
@ -3,7 +3,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/FieldToDataType.h>
|
||||
#include <Formats/BlockInputStreamFromRowInputStream.h>
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Functions/FunctionsConversion.h>
|
||||
#include <Functions/FunctionFactory.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());
|
||||
|
||||
Tokens tokens_number(istr.position(), istr.buffer().end());
|
||||
TokenIterator iterator(tokens_number);
|
||||
IParser::Pos iterator(tokens_number);
|
||||
Expected expected;
|
||||
ASTPtr ast;
|
||||
if (nullable && parser_null.parse(iterator, ast, expected))
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Formats/ValuesBlockInputStream.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/BlockInputStreamFromRowInputStream.h>
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -33,8 +33,8 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
ValuesBlockInputStream::ValuesBlockInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_,
|
||||
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),
|
||||
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)
|
||||
{
|
||||
@ -48,8 +48,14 @@ Block ValuesBlockInputStream::readImpl()
|
||||
{
|
||||
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
|
||||
{
|
||||
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
|
||||
Tokens tokens(istr.position(), istr.buffer().end());
|
||||
TokenIterator token_iterator(tokens);
|
||||
IParser::Pos token_iterator(tokens);
|
||||
|
||||
ASTPtr ast;
|
||||
if (!parser.parse(token_iterator, ast, expected))
|
||||
@ -276,9 +282,11 @@ void registerInputFormatValues(FormatFactory & factory)
|
||||
const Block & sample,
|
||||
const Context & context,
|
||||
UInt64 max_block_size,
|
||||
UInt64 rows_portion_size,
|
||||
FormatFactory::ReadCallback,
|
||||
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);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Formats/ConstantExpressionTemplate.h>
|
||||
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -23,7 +24,7 @@ public:
|
||||
* 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).
|
||||
*/
|
||||
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"; }
|
||||
Block getHeader() const override { return header; }
|
||||
@ -50,6 +51,7 @@ private:
|
||||
std::unique_ptr<Context> context; /// pimpl
|
||||
const FormatSettings format_settings;
|
||||
UInt64 max_block_size;
|
||||
UInt64 rows_portion_size;
|
||||
size_t num_columns;
|
||||
size_t total_rows = 0;
|
||||
|
||||
|
@ -36,6 +36,8 @@ struct RowInputFormatParams
|
||||
OverflowMode timeout_overflow_mode = OverflowMode::THROW;
|
||||
};
|
||||
|
||||
bool isParseError(int code);
|
||||
|
||||
///Row oriented input format: reads data row by row.
|
||||
class IRowInputFormat : public IInputFormat
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user