mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Improve performance of format Regexp
This commit is contained in:
parent
678dd6b3f2
commit
872d0a0fbe
@ -45,7 +45,7 @@ bool RegexpFieldExtractor::parseRow(PeekableReadBuffer & buf)
|
|||||||
buf.makeContinuousMemoryFromCheckpointToPos();
|
buf.makeContinuousMemoryFromCheckpointToPos();
|
||||||
buf.rollbackToCheckpoint();
|
buf.rollbackToCheckpoint();
|
||||||
|
|
||||||
bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
|
bool match = re2_st::RE2::FullMatchN(re2_st::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
|
||||||
|
|
||||||
if (!match && !skip_unmatched)
|
if (!match && !skip_unmatched)
|
||||||
throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
|
throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <re2/re2.h>
|
#include <re2_st/re2.h>
|
||||||
#include <re2/stringpiece.h>
|
#include <re2_st/stringpiece.h>
|
||||||
#include <string>
|
#include <string>
|
||||||
#include <vector>
|
#include <vector>
|
||||||
#include <Core/Block.h>
|
#include <Core/Block.h>
|
||||||
@ -12,6 +12,7 @@
|
|||||||
#include <IO/PeekableReadBuffer.h>
|
#include <IO/PeekableReadBuffer.h>
|
||||||
#include <Formats/ParsedTemplateFormatString.h>
|
#include <Formats/ParsedTemplateFormatString.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
@ -26,17 +27,17 @@ public:
|
|||||||
/// Return true if row was successfully parsed and row fields were extracted.
|
/// Return true if row was successfully parsed and row fields were extracted.
|
||||||
bool parseRow(PeekableReadBuffer & buf);
|
bool parseRow(PeekableReadBuffer & buf);
|
||||||
|
|
||||||
re2::StringPiece getField(size_t index) { return matched_fields[index]; }
|
re2_st::StringPiece getField(size_t index) { return matched_fields[index]; }
|
||||||
size_t getMatchedFieldsSize() const { return matched_fields.size(); }
|
size_t getMatchedFieldsSize() const { return matched_fields.size(); }
|
||||||
size_t getNumberOfGroups() const { return regexp.NumberOfCapturingGroups(); }
|
size_t getNumberOfGroups() const { return regexp.NumberOfCapturingGroups(); }
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const RE2 regexp;
|
const re2_st::RE2 regexp;
|
||||||
// The vector of fields extracted from line using regexp.
|
// The vector of fields extracted from line using regexp.
|
||||||
std::vector<re2::StringPiece> matched_fields;
|
std::vector<re2_st::StringPiece> matched_fields;
|
||||||
// These two vectors are needed to use RE2::FullMatchN (function for extracting fields).
|
// These two vectors are needed to use RE2::FullMatchN (function for extracting fields).
|
||||||
std::vector<RE2::Arg> re2_arguments;
|
std::vector<re2_st::RE2::Arg> re2_arguments;
|
||||||
std::vector<RE2::Arg *> re2_arguments_ptrs;
|
std::vector<re2_st::RE2::Arg *> re2_arguments_ptrs;
|
||||||
bool skip_unmatched;
|
bool skip_unmatched;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -26,7 +26,6 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS)
|
|||||||
add_subdirectory (zookeeper-adjust-block-numbers-to-parts)
|
add_subdirectory (zookeeper-adjust-block-numbers-to-parts)
|
||||||
add_subdirectory (wikistat-loader)
|
add_subdirectory (wikistat-loader)
|
||||||
add_subdirectory (check-marks)
|
add_subdirectory (check-marks)
|
||||||
add_subdirectory (convert-month-partitioned-parts)
|
|
||||||
add_subdirectory (checksum-for-compressed-block)
|
add_subdirectory (checksum-for-compressed-block)
|
||||||
add_subdirectory (db-generator)
|
add_subdirectory (db-generator)
|
||||||
add_subdirectory (wal-dump)
|
add_subdirectory (wal-dump)
|
||||||
|
@ -1,2 +0,0 @@
|
|||||||
add_executable (convert-month-partitioned-parts main.cpp)
|
|
||||||
target_link_libraries(convert-month-partitioned-parts PRIVATE clickhouse_aggregate_functions dbms clickhouse_parsers boost::program_options)
|
|
@ -1,148 +0,0 @@
|
|||||||
#include <DataTypes/DataTypeDate.h>
|
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
|
||||||
#include <IO/HashingWriteBuffer.h>
|
|
||||||
#include <IO/ReadBufferFromFile.h>
|
|
||||||
#include <IO/WriteBufferFromFile.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
|
||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Storages/MergeTree/IMergeTreeDataPart.h>
|
|
||||||
#include <Storages/MergeTree/MergeTreePartInfo.h>
|
|
||||||
#include <Storages/MergeTree/localBackup.h>
|
|
||||||
#include <Common/Exception.h>
|
|
||||||
|
|
||||||
#include <boost/program_options.hpp>
|
|
||||||
#include <Poco/Path.h>
|
|
||||||
#include <Poco/File.h>
|
|
||||||
|
|
||||||
#include <iostream>
|
|
||||||
#include <Disks/DiskLocal.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int DIRECTORY_ALREADY_EXISTS;
|
|
||||||
extern const int BAD_DATA_PART_NAME;
|
|
||||||
extern const int NO_FILE_IN_DATA_PART;
|
|
||||||
}
|
|
||||||
|
|
||||||
void run(String part_path, String date_column, String dest_path)
|
|
||||||
{
|
|
||||||
std::shared_ptr<IDisk> disk = std::make_shared<DiskLocal>("local", "/", 0);
|
|
||||||
auto old_part_path = Poco::Path::forDirectory(part_path);
|
|
||||||
const String & old_part_name = old_part_path.directory(old_part_path.depth() - 1);
|
|
||||||
String old_part_path_str = old_part_path.toString();
|
|
||||||
|
|
||||||
auto part_info = MergeTreePartInfo::fromPartName(old_part_name, MergeTreeDataFormatVersion(0));
|
|
||||||
String new_part_name = part_info.getPartName();
|
|
||||||
|
|
||||||
auto new_part_path = Poco::Path::forDirectory(dest_path);
|
|
||||||
new_part_path.pushDirectory(new_part_name);
|
|
||||||
if (Poco::File(new_part_path).exists())
|
|
||||||
throw Exception("Destination part directory `" + new_part_path.toString() + "` already exists",
|
|
||||||
ErrorCodes::DIRECTORY_ALREADY_EXISTS);
|
|
||||||
|
|
||||||
DayNum min_date;
|
|
||||||
DayNum max_date;
|
|
||||||
MergeTreePartInfo::parseMinMaxDatesFromPartName(old_part_name, min_date, max_date);
|
|
||||||
|
|
||||||
const auto & time_zone = DateLUT::instance();
|
|
||||||
UInt32 yyyymm = time_zone.toNumYYYYMM(min_date);
|
|
||||||
if (yyyymm != time_zone.toNumYYYYMM(max_date))
|
|
||||||
throw Exception("Part " + old_part_name + " spans different months",
|
|
||||||
ErrorCodes::BAD_DATA_PART_NAME);
|
|
||||||
|
|
||||||
ReadBufferFromFile checksums_in(old_part_path_str + "checksums.txt", 4096);
|
|
||||||
MergeTreeDataPartChecksums checksums;
|
|
||||||
checksums.read(checksums_in);
|
|
||||||
|
|
||||||
auto date_col_checksum_it = checksums.files.find(date_column + ".bin");
|
|
||||||
if (date_col_checksum_it == checksums.files.end())
|
|
||||||
throw Exception("Couldn't find checksum for the date column .bin file `" + date_column + ".bin`",
|
|
||||||
ErrorCodes::NO_FILE_IN_DATA_PART);
|
|
||||||
|
|
||||||
UInt64 rows = date_col_checksum_it->second.uncompressed_size / DataTypeDate().getSizeOfValueInMemory();
|
|
||||||
|
|
||||||
auto new_tmp_part_path = Poco::Path::forDirectory(dest_path);
|
|
||||||
new_tmp_part_path.pushDirectory("tmp_convert_" + new_part_name);
|
|
||||||
String new_tmp_part_path_str = new_tmp_part_path.toString();
|
|
||||||
try
|
|
||||||
{
|
|
||||||
Poco::File(new_tmp_part_path).remove(/* recursive = */ true);
|
|
||||||
}
|
|
||||||
catch (const Poco::FileNotFoundException &)
|
|
||||||
{
|
|
||||||
/// If the file is already deleted, do nothing.
|
|
||||||
}
|
|
||||||
localBackup(disk, old_part_path.toString(), new_tmp_part_path.toString(), {});
|
|
||||||
|
|
||||||
WriteBufferFromFile count_out(new_tmp_part_path_str + "count.txt", 4096);
|
|
||||||
HashingWriteBuffer count_out_hashing(count_out);
|
|
||||||
writeIntText(rows, count_out_hashing);
|
|
||||||
count_out_hashing.next();
|
|
||||||
checksums.files["count.txt"].file_size = count_out_hashing.count();
|
|
||||||
checksums.files["count.txt"].file_hash = count_out_hashing.getHash();
|
|
||||||
|
|
||||||
IMergeTreeDataPart::MinMaxIndex minmax_idx(min_date, max_date);
|
|
||||||
Names minmax_idx_columns = {date_column};
|
|
||||||
DataTypes minmax_idx_column_types = {std::make_shared<DataTypeDate>()};
|
|
||||||
minmax_idx.store(minmax_idx_columns, minmax_idx_column_types, disk, new_tmp_part_path_str, checksums);
|
|
||||||
|
|
||||||
Block partition_key_sample{{nullptr, std::make_shared<DataTypeUInt32>(), makeASTFunction("toYYYYMM", std::make_shared<ASTIdentifier>(date_column))->getColumnName()}};
|
|
||||||
|
|
||||||
MergeTreePartition partition(yyyymm);
|
|
||||||
partition.store(partition_key_sample, disk, new_tmp_part_path_str, checksums);
|
|
||||||
String partition_id = partition.getID(partition_key_sample);
|
|
||||||
|
|
||||||
Poco::File(new_tmp_part_path_str + "checksums.txt").setWriteable();
|
|
||||||
WriteBufferFromFile checksums_out(new_tmp_part_path_str + "checksums.txt", 4096);
|
|
||||||
checksums.write(checksums_out);
|
|
||||||
checksums_in.close();
|
|
||||||
checksums_out.close();
|
|
||||||
|
|
||||||
Poco::File(new_tmp_part_path).renameTo(new_part_path.toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
int main(int argc, char ** argv)
|
|
||||||
try
|
|
||||||
{
|
|
||||||
boost::program_options::options_description desc("Allowed options");
|
|
||||||
desc.add_options()
|
|
||||||
("help,h", "produce help message")
|
|
||||||
("part", boost::program_options::value<std::string>()->required(),
|
|
||||||
"part directory to convert")
|
|
||||||
("date-column", boost::program_options::value<std::string>()->required(),
|
|
||||||
"name of the date column")
|
|
||||||
("to", boost::program_options::value<std::string>()->required(),
|
|
||||||
"destination directory")
|
|
||||||
;
|
|
||||||
|
|
||||||
boost::program_options::variables_map options;
|
|
||||||
boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options);
|
|
||||||
|
|
||||||
if (options.count("help") || options.size() < 3)
|
|
||||||
{
|
|
||||||
std::cout
|
|
||||||
<< "Convert a MergeTree part from the old-style month-partitioned table "
|
|
||||||
<< "(e.g. 20140317_20140323_2_2_0) to the format suitable for ATTACH'ing to a custom-partitioned "
|
|
||||||
<< "table (201403_2_2_0)." << std::endl << std::endl;
|
|
||||||
std::cout << desc << std::endl;
|
|
||||||
return 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
auto part_path = options.at("part").as<DB::String>();
|
|
||||||
auto date_column = options.at("date-column").as<DB::String>();
|
|
||||||
auto dest_path = options.at("to").as<DB::String>();
|
|
||||||
|
|
||||||
DB::run(part_path, date_column, dest_path);
|
|
||||||
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
std::cerr << DB::getCurrentExceptionMessage(true) << '\n';
|
|
||||||
throw;
|
|
||||||
}
|
|
Loading…
Reference in New Issue
Block a user