mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Remove ValuesRowInputStream.
This commit is contained in:
parent
e6b3f3f03a
commit
1f6a16b03a
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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));
|
||||
|
Loading…
Reference in New Issue
Block a user