mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 03:22:14 +00:00
97f2a2213e
* Move some code outside dbms/src folder * Fix paths
196 lines
6.5 KiB
C++
196 lines
6.5 KiB
C++
#include <stdlib.h>
|
|
#include <common/find_symbols.h>
|
|
#include <Processors/Formats/Impl/RegexpRowInputFormat.h>
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
namespace DB
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
{
|
|
extern const int INCORRECT_DATA;
|
|
extern const int BAD_ARGUMENTS;
|
|
}
|
|
|
|
RegexpRowInputFormat::RegexpRowInputFormat(
|
|
ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
|
|
: IRowInputFormat(header_, in_, std::move(params_)), buf(in_), format_settings(format_settings_), regexp(format_settings_.regexp.regexp)
|
|
{
|
|
size_t fields_count = regexp.NumberOfCapturingGroups();
|
|
matched_fields.resize(fields_count);
|
|
re2_arguments.resize(fields_count);
|
|
re2_arguments_ptrs.resize(fields_count);
|
|
for (size_t i = 0; i != fields_count; ++i)
|
|
{
|
|
// Bind an argument to a matched field.
|
|
re2_arguments[i] = &matched_fields[i];
|
|
// Save pointer to argument.
|
|
re2_arguments_ptrs[i] = &re2_arguments[i];
|
|
}
|
|
|
|
field_format = stringToFormat(format_settings_.regexp.escaping_rule);
|
|
}
|
|
|
|
RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const String & format)
|
|
{
|
|
if (format == "Escaped")
|
|
return ColumnFormat::Escaped;
|
|
if (format == "Quoted")
|
|
return ColumnFormat::Quoted;
|
|
if (format == "CSV")
|
|
return ColumnFormat::Csv;
|
|
if (format == "JSON")
|
|
return ColumnFormat::Json;
|
|
throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS);
|
|
}
|
|
|
|
bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
|
|
{
|
|
const auto & type = getPort().getHeader().getByPosition(index).type;
|
|
bool parse_as_nullable = format_settings.null_as_default && !type->isNullable();
|
|
bool read = true;
|
|
ReadBuffer field_buf(const_cast<char *>(matched_fields[index].data()), matched_fields[index].size(), 0);
|
|
try
|
|
{
|
|
switch (field_format)
|
|
{
|
|
case ColumnFormat::Escaped:
|
|
if (parse_as_nullable)
|
|
read = DataTypeNullable::deserializeTextEscaped(*columns[index], field_buf, format_settings, type);
|
|
else
|
|
type->deserializeAsTextEscaped(*columns[index], field_buf, format_settings);
|
|
break;
|
|
case ColumnFormat::Quoted:
|
|
if (parse_as_nullable)
|
|
read = DataTypeNullable::deserializeTextQuoted(*columns[index], field_buf, format_settings, type);
|
|
else
|
|
type->deserializeAsTextQuoted(*columns[index], field_buf, format_settings);
|
|
break;
|
|
case ColumnFormat::Csv:
|
|
if (parse_as_nullable)
|
|
read = DataTypeNullable::deserializeTextCSV(*columns[index], field_buf, format_settings, type);
|
|
else
|
|
type->deserializeAsTextCSV(*columns[index], field_buf, format_settings);
|
|
break;
|
|
case ColumnFormat::Json:
|
|
if (parse_as_nullable)
|
|
read = DataTypeNullable::deserializeTextJSON(*columns[index], field_buf, format_settings, type);
|
|
else
|
|
type->deserializeAsTextJSON(*columns[index], field_buf, format_settings);
|
|
break;
|
|
default:
|
|
break;
|
|
}
|
|
}
|
|
catch (Exception & e)
|
|
{
|
|
e.addMessage("(while read the value of column " + getPort().getHeader().getByPosition(index).name + ")");
|
|
throw;
|
|
}
|
|
return read;
|
|
}
|
|
|
|
void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext)
|
|
{
|
|
if (matched_fields.size() != columns.size())
|
|
throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA);
|
|
|
|
ext.read_columns.assign(columns.size(), false);
|
|
for (size_t columns_index = 0; columns_index < columns.size(); ++columns_index)
|
|
{
|
|
ext.read_columns[columns_index] = readField(columns_index, columns);
|
|
}
|
|
}
|
|
|
|
bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
|
|
{
|
|
if (buf.eof())
|
|
return false;
|
|
|
|
PeekableReadBufferCheckpoint checkpoint{buf};
|
|
|
|
size_t line_size = 0;
|
|
|
|
do
|
|
{
|
|
char * pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end());
|
|
line_size += pos - buf.position();
|
|
buf.position() = pos;
|
|
} while (buf.position() == buf.buffer().end() && !buf.eof());
|
|
|
|
buf.makeContinuousMemoryFromCheckpointToPos();
|
|
buf.rollbackToCheckpoint();
|
|
|
|
bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
|
|
bool read_line = true;
|
|
|
|
if (!match)
|
|
{
|
|
if (!format_settings.regexp.skip_unmatched)
|
|
throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
|
|
read_line = false;
|
|
}
|
|
|
|
if (read_line)
|
|
readFieldsFromMatch(columns, ext);
|
|
|
|
buf.position() += line_size;
|
|
|
|
checkChar('\r', buf);
|
|
if (!buf.eof() && !checkChar('\n', buf))
|
|
throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA);
|
|
|
|
return true;
|
|
}
|
|
|
|
void registerInputFormatProcessorRegexp(FormatFactory & factory)
|
|
{
|
|
factory.registerInputFormatProcessor("Regexp", [](
|
|
ReadBuffer & buf,
|
|
const Block & sample,
|
|
IRowInputFormat::Params params,
|
|
const FormatSettings & settings)
|
|
{
|
|
return std::make_shared<RegexpRowInputFormat>(buf, sample, std::move(params), settings);
|
|
});
|
|
}
|
|
|
|
static bool fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
|
{
|
|
char * pos = in.position();
|
|
bool need_more_data = true;
|
|
|
|
while (loadAtPosition(in, memory, pos) && need_more_data)
|
|
{
|
|
pos = find_first_symbols<'\n', '\r'>(pos, in.buffer().end());
|
|
if (pos == in.buffer().end())
|
|
continue;
|
|
|
|
// Support DOS-style newline ("\r\n")
|
|
if (*pos == '\r')
|
|
{
|
|
++pos;
|
|
if (pos == in.buffer().end())
|
|
loadAtPosition(in, memory, pos);
|
|
}
|
|
|
|
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
|
|
need_more_data = false;
|
|
|
|
++pos;
|
|
|
|
}
|
|
|
|
saveUpToPosition(in, memory, pos);
|
|
|
|
return loadAtPosition(in, memory, pos);
|
|
}
|
|
|
|
void registerFileSegmentationEngineRegexp(FormatFactory & factory)
|
|
{
|
|
factory.registerFileSegmentationEngine("Regexp", &fileSegmentationEngineRegexpImpl);
|
|
}
|
|
|
|
}
|