2022-05-08 17:01:47 +00:00
|
|
|
#include <cstdlib>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/find_symbols.h>
|
2020-02-14 19:48:45 +00:00
|
|
|
#include <Processors/Formats/Impl/RegexpRowInputFormat.h>
|
2021-03-09 14:46:52 +00:00
|
|
|
#include <DataTypes/Serializations/SerializationNullable.h>
|
2021-11-09 13:14:07 +00:00
|
|
|
#include <Formats/EscapingRuleUtils.h>
|
2022-10-28 18:44:36 +00:00
|
|
|
#include <Formats/newLineSegmentationEngine.h>
|
2020-02-14 19:48:45 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int INCORRECT_DATA;
|
2021-02-15 13:10:14 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2020-02-14 19:48:45 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
RegexpFieldExtractor::RegexpFieldExtractor(const FormatSettings & format_settings) : regexp(format_settings.regexp.regexp), skip_unmatched(format_settings.regexp.skip_unmatched)
|
2020-02-14 19:48:45 +00:00
|
|
|
{
|
2020-02-21 15:21:31 +00:00
|
|
|
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];
|
|
|
|
}
|
2020-06-11 00:51:27 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
bool RegexpFieldExtractor::parseRow(PeekableReadBuffer & buf)
|
|
|
|
{
|
|
|
|
PeekableReadBufferCheckpoint checkpoint{buf};
|
|
|
|
|
|
|
|
size_t line_size = 0;
|
|
|
|
|
|
|
|
do
|
|
|
|
{
|
2022-01-31 23:49:26 +00:00
|
|
|
char * pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end());
|
2021-12-15 11:30:57 +00:00
|
|
|
line_size += pos - buf.position();
|
|
|
|
buf.position() = pos;
|
|
|
|
} while (buf.position() == buf.buffer().end() && !buf.eof());
|
|
|
|
|
|
|
|
buf.makeContinuousMemoryFromCheckpointToPos();
|
|
|
|
buf.rollbackToCheckpoint();
|
|
|
|
|
2022-01-31 23:49:26 +00:00
|
|
|
/// Allow DOS line endings.
|
|
|
|
size_t line_to_match = line_size;
|
|
|
|
if (line_size > 0 && buf.position()[line_size - 1] == '\r')
|
|
|
|
--line_to_match;
|
|
|
|
|
2022-10-07 10:46:45 +00:00
|
|
|
bool match = re2_st::RE2::FullMatchN(
|
|
|
|
re2_st::StringPiece(buf.position(), line_to_match),
|
|
|
|
regexp,
|
|
|
|
re2_arguments_ptrs.data(),
|
|
|
|
static_cast<int>(re2_arguments_ptrs.size()));
|
2021-12-15 11:30:57 +00:00
|
|
|
|
|
|
|
if (!match && !skip_unmatched)
|
2022-01-31 23:49:26 +00:00
|
|
|
throw Exception("Line \"" + std::string(buf.position(), line_to_match) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
|
2021-12-15 11:30:57 +00:00
|
|
|
|
|
|
|
buf.position() += line_size;
|
|
|
|
if (!buf.eof() && !checkChar('\n', buf))
|
2022-01-31 23:49:26 +00:00
|
|
|
throw Exception("No \\n at the end of line.", ErrorCodes::LOGICAL_ERROR);
|
2021-12-15 11:30:57 +00:00
|
|
|
|
|
|
|
return match;
|
|
|
|
}
|
|
|
|
|
|
|
|
RegexpRowInputFormat::RegexpRowInputFormat(
|
|
|
|
ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
|
|
|
|
: RegexpRowInputFormat(std::make_unique<PeekableReadBuffer>(in_), header_, params_, format_settings_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
RegexpRowInputFormat::RegexpRowInputFormat(
|
2022-01-31 23:49:26 +00:00
|
|
|
std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, Params params_, const FormatSettings & format_settings_)
|
|
|
|
: IRowInputFormat(header_, *buf_, std::move(params_))
|
|
|
|
, buf(std::move(buf_))
|
|
|
|
, format_settings(format_settings_)
|
|
|
|
, escaping_rule(format_settings_.regexp.escaping_rule)
|
|
|
|
, field_extractor(RegexpFieldExtractor(format_settings_))
|
2021-12-15 11:30:57 +00:00
|
|
|
{
|
|
|
|
}
|
2020-02-21 15:21:31 +00:00
|
|
|
|
2020-06-11 00:51:27 +00:00
|
|
|
void RegexpRowInputFormat::resetParser()
|
|
|
|
{
|
|
|
|
IRowInputFormat::resetParser();
|
2021-12-10 17:54:08 +00:00
|
|
|
buf->reset();
|
2020-02-14 19:48:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
|
|
|
|
{
|
|
|
|
const auto & type = getPort().getHeader().getByPosition(index).type;
|
2021-12-15 11:30:57 +00:00
|
|
|
auto matched_field = field_extractor.getField(index);
|
|
|
|
ReadBuffer field_buf(const_cast<char *>(matched_field.data()), matched_field.size(), 0);
|
2020-02-14 19:48:45 +00:00
|
|
|
try
|
|
|
|
{
|
2021-11-09 13:14:07 +00:00
|
|
|
return deserializeFieldByEscapingRule(type, serializations[index], *columns[index], field_buf, escaping_rule, format_settings);
|
2020-02-14 19:48:45 +00:00
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
2020-11-19 14:44:58 +00:00
|
|
|
e.addMessage("(while reading the value of column " + getPort().getHeader().getByPosition(index).name + ")");
|
2020-02-14 19:48:45 +00:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext)
|
|
|
|
{
|
2021-12-15 11:30:57 +00:00
|
|
|
if (field_extractor.getMatchedFieldsSize() != columns.size())
|
2020-02-14 19:48:45 +00:00
|
|
|
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)
|
|
|
|
{
|
2021-12-10 17:54:08 +00:00
|
|
|
if (buf->eof())
|
2020-02-14 19:48:45 +00:00
|
|
|
return false;
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
if (field_extractor.parseRow(*buf))
|
2020-02-21 15:21:31 +00:00
|
|
|
readFieldsFromMatch(columns, ext);
|
2020-02-14 19:48:45 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-12-10 17:54:08 +00:00
|
|
|
void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_)
|
|
|
|
{
|
|
|
|
buf = std::make_unique<PeekableReadBuffer>(in_);
|
|
|
|
IInputFormat::setReadBuffer(*buf);
|
|
|
|
}
|
|
|
|
|
2022-03-24 12:54:12 +00:00
|
|
|
RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
|
2021-12-15 11:30:57 +00:00
|
|
|
: IRowSchemaReader(
|
|
|
|
buf,
|
2022-03-24 12:54:12 +00:00
|
|
|
format_settings_,
|
2021-12-15 11:30:57 +00:00
|
|
|
getDefaultDataTypeForEscapingRule(format_settings_.regexp.escaping_rule))
|
|
|
|
, field_extractor(format_settings)
|
|
|
|
, buf(in_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
DataTypes RegexpSchemaReader::readRowAndGetDataTypes()
|
|
|
|
{
|
|
|
|
if (buf.eof())
|
|
|
|
return {};
|
|
|
|
|
|
|
|
field_extractor.parseRow(buf);
|
|
|
|
|
|
|
|
DataTypes data_types;
|
|
|
|
data_types.reserve(field_extractor.getMatchedFieldsSize());
|
|
|
|
for (size_t i = 0; i != field_extractor.getMatchedFieldsSize(); ++i)
|
|
|
|
{
|
|
|
|
String field(field_extractor.getField(i));
|
2022-03-24 12:54:12 +00:00
|
|
|
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule));
|
2021-12-15 11:30:57 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return data_types;
|
|
|
|
}
|
|
|
|
|
2022-07-13 15:57:55 +00:00
|
|
|
void RegexpSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
|
|
|
|
{
|
|
|
|
transformInferredTypesIfNeeded(type, new_type, format_settings, format_settings.regexp.escaping_rule);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-10-11 16:11:50 +00:00
|
|
|
void registerInputFormatRegexp(FormatFactory & factory)
|
2020-02-14 19:48:45 +00:00
|
|
|
{
|
2021-10-11 16:11:50 +00:00
|
|
|
factory.registerInputFormat("Regexp", [](
|
2020-02-14 19:48:45 +00:00
|
|
|
ReadBuffer & buf,
|
|
|
|
const Block & sample,
|
|
|
|
IRowInputFormat::Params params,
|
|
|
|
const FormatSettings & settings)
|
|
|
|
{
|
|
|
|
return std::make_shared<RegexpRowInputFormat>(buf, sample, std::move(params), settings);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
void registerFileSegmentationEngineRegexp(FormatFactory & factory)
|
|
|
|
{
|
2022-10-28 18:44:36 +00:00
|
|
|
factory.registerFileSegmentationEngine("Regexp", &newLineFileSegmentationEngine);
|
2020-02-14 19:48:45 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
void registerRegexpSchemaReader(FormatFactory & factory)
|
|
|
|
{
|
2022-03-24 12:54:12 +00:00
|
|
|
factory.registerSchemaReader("Regexp", [](ReadBuffer & buf, const FormatSettings & settings)
|
2021-12-15 11:30:57 +00:00
|
|
|
{
|
2022-03-24 12:54:12 +00:00
|
|
|
return std::make_shared<RegexpSchemaReader>(buf, settings);
|
2021-12-15 11:30:57 +00:00
|
|
|
});
|
2022-08-19 16:39:13 +00:00
|
|
|
factory.registerAdditionalInfoForSchemaCacheGetter("Regexp", [](const FormatSettings & settings)
|
|
|
|
{
|
|
|
|
auto result = getAdditionalFormatInfoByEscapingRule(settings, settings.regexp.escaping_rule);
|
|
|
|
return result + fmt::format(", regexp={}", settings.regexp.regexp);
|
|
|
|
});
|
2021-12-15 11:30:57 +00:00
|
|
|
}
|
|
|
|
|
2020-02-14 19:48:45 +00:00
|
|
|
}
|