Remove ValuesRowInputStream.

This commit is contained in:
Nikolai Kochetov 2019-08-02 20:16:58 +03:00
parent e6b3f3f03a
commit 1f6a16b03a
6 changed files with 6 additions and 237 deletions

View File

@ -200,7 +200,6 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro
void registerInputFormatNative(FormatFactory & factory);
void registerOutputFormatNative(FormatFactory & factory);
void registerInputFormatTabSeparated(FormatFactory & factory);
void registerInputFormatValues(FormatFactory & factory);
void registerOutputFormatValues(FormatFactory & factory);
void registerInputFormatCSV(FormatFactory & factory);
@ -249,7 +248,6 @@ FormatFactory::FormatFactory()
registerInputFormatNative(*this);
registerOutputFormatNative(*this);
registerInputFormatTabSeparated(*this);
registerInputFormatValues(*this);
registerOutputFormatValues(*this);
registerInputFormatCSV(*this);

View File

@ -1,25 +0,0 @@
#pragma once
#include <Formats/TabSeparatedRowOutputStream.h>
namespace DB
{
struct FormatSettings;
/** A stream for outputting data in tsv format, but without escaping individual values.
* (That is, the output is irreversible.)
*/
class TabSeparatedRawRowOutputStream : public TabSeparatedRowOutputStream
{
public:
TabSeparatedRawRowOutputStream(WriteBuffer & ostr_, const Block & sample_, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: TabSeparatedRowOutputStream(ostr_, sample_, with_names_, with_types_, format_settings_) {}
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override
{
type.serializeAsText(column, row_num, ostr, format_settings);
}
};
}

View File

@ -1,168 +0,0 @@
#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/ValuesRowInputStream.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;
}
ValuesRowInputStream::ValuesRowInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, const FormatSettings & format_settings)
: istr(istr_), header(header_), context(std::make_unique<Context>(context_)), format_settings(format_settings)
{
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
skipBOMIfExists(istr);
}
bool ValuesRowInputStream::read(MutableColumns & columns, RowReadExtension &)
{
size_t num_columns = columns.size();
skipWhitespaceIfAny(istr);
if (istr.eof() || *istr.position() == ';')
return false;
/** Typically, this is the usual format for streaming parsing.
* But as an exception, it also supports processing arbitrary expressions instead of values.
* This is very inefficient. But if there are no expressions, then there is no overhead.
*/
ParserExpression parser;
assertChar('(', istr);
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespaceIfAny(istr);
char * prev_istr_position = istr.position();
size_t prev_istr_bytes = istr.count() - istr.offset();
bool rollback_on_exception = false;
try
{
header.getByPosition(i).type->deserializeAsTextQuoted(*columns[i], istr, format_settings);
rollback_on_exception = true;
skipWhitespaceIfAny(istr);
if (i != num_columns - 1)
assertChar(',', istr);
else
assertChar(')', istr);
}
catch (const Exception & e)
{
if (!format_settings.values.interpret_expressions)
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)
{
/// TODO Case when the expression does not fit entirely in the buffer.
/// If the beginning of the value is no longer in the buffer.
if (istr.count() - istr.offset() != prev_istr_bytes)
throw;
if (rollback_on_exception)
columns[i]->popBack(1);
const IDataType & type = *header.getByPosition(i).type;
Expected expected;
Tokens tokens(prev_istr_position, istr.buffer().end());
TokenIterator token_iterator(tokens);
ASTPtr ast;
if (!parser.parse(token_iterator, ast, expected))
throw Exception("Cannot parse expression of type " + type.getName() + " here: "
+ String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),
ErrorCodes::SYNTAX_ERROR);
istr.position() = const_cast<char *>(token_iterator->begin);
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())
{
if (!type.isNullable())
throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value)
+ ", that is out of range of type " + type.getName()
+ ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE};
}
columns[i]->insert(value);
skipWhitespaceIfAny(istr);
if (i != num_columns - 1)
assertChar(',', istr);
else
assertChar(')', istr);
}
else
throw;
}
}
skipWhitespaceIfAny(istr);
if (!istr.eof() && *istr.position() == ',')
++istr.position();
return true;
}
void registerInputFormatValues(FormatFactory & factory)
{
factory.registerInputFormat("Values", [](
ReadBuffer & buf,
const Block & sample,
const Context & context,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<ValuesRowInputStream>(buf, sample, context, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
}

View File

@ -1,35 +0,0 @@
#pragma once
#include <Core/Block.h>
#include <Formats/IRowInputStream.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class Context;
class ReadBuffer;
/** Stream to read data in VALUES format (as in INSERT query).
*/
class ValuesRowInputStream : public IRowInputStream
{
public:
/** Data is parsed using fast, streaming parser.
* 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).
*/
ValuesRowInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, const FormatSettings & format_settings);
bool read(MutableColumns & columns, RowReadExtension &) override;
private:
ReadBuffer & istr;
Block header;
std::unique_ptr<Context> context; /// pimpl
const FormatSettings format_settings;
};
}

View File

@ -159,7 +159,7 @@ void registerInputFormatProcessorValues(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<ValuesRowInputFormat>(buf, sample, params, context, settings);
return std::make_shared<ValuesRowInputFormat>(buf, sample, std::move(params), context, settings);
});
}

View File

@ -17,7 +17,7 @@
#include <Parsers/queryToString.h>
#include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/MarkInCompressedFile.h>
#include <Formats/ValuesRowInputStream.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/copyData.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromString.h>
@ -2488,17 +2488,16 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context
ReadBufferFromMemory right_paren_buf(")", 1);
ConcatReadBuffer buf({&left_paren_buf, &fields_buf, &right_paren_buf});
ValuesRowInputStream input_stream(buf, partition_key_sample, context, format_settings);
MutableColumns columns = partition_key_sample.cloneEmptyColumns();
auto input_stream = FormatFactory::instance().getInput("Values", buf, partition_key_sample, context, context.getSettingsRef().max_block_size);
RowReadExtension unused;
if (!input_stream.read(columns, unused))
auto block = input_stream->read();
if (!block || !block.rows())
throw Exception(
"Could not parse partition value: `" + partition_ast.fields_str.toString() + "`",
ErrorCodes::INVALID_PARTITION_VALUE);
for (size_t i = 0; i < fields_count; ++i)
columns[i]->get(0, partition_row[i]);
block.getByPosition(i).column->get(0, partition_row[i]);
}
MergeTreePartition partition(std::move(partition_row));