Merge pull request #58458 from rschu1ze/replace-std_regexp-by-re2

Replace `std::regex` by re2
This commit is contained in:
Robert Schulze 2024-01-10 17:46:54 +01:00 committed by GitHub
commit 52d75ab68f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
53 changed files with 139 additions and 384 deletions

View File

@ -17,15 +17,7 @@
#include <Common/Config/ConfigProcessor.h>
#include <Common/Exception.h>
#include <Common/parseGlobs.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
static void setupLogging(const std::string & log_level)
{

View File

@ -16,6 +16,7 @@
#include <Common/SipHash.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/ShellCommand.h>
#include <Common/re2.h>
#include <base/find_symbols.h>
#include <IO/copyData.h>
@ -24,15 +25,6 @@
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
static constexpr auto documentation = R"(
A tool to extract information from Git repository for analytics.

View File

@ -1,5 +1,6 @@
#include <Common/Exception.h>
#include <Common/TerminalSize.h>
#include <Common/re2.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromFile.h>
@ -12,15 +13,6 @@
#include <boost/program_options.hpp>
#include <filesystem>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace fs = std::filesystem;
#define EXTRACT_PATH_PATTERN ".*\\/store/(.*)"

View File

@ -24,20 +24,12 @@
#include <Storages/MergeTree/MergeTreeSettings.h>
#include <base/defines.h>
#include <IO/Operators.h>
#include <Common/re2.h>
#include <Poco/AccessExpireCache.h>
#include <boost/algorithm/string/join.hpp>
#include <filesystem>
#include <mutex>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{
namespace ErrorCodes

View File

@ -3,15 +3,7 @@
#include <Analyzer/Identifier.h>
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ListNode.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
namespace DB
{

View File

@ -4,15 +4,7 @@
#include <Analyzer/IQueryTreeNode.h>
#include <Analyzer/ColumnTransformers.h>
#include <Parsers/ASTAsterisk.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
namespace DB
{

View File

@ -2,6 +2,7 @@
#include <Common/DateLUT.h>
#include <Common/Exception.h>
#include <Common/re2.h>
#include <chrono>
#include <filesystem>
@ -11,15 +12,6 @@
#include <boost/algorithm/string.hpp>
#include <boost/algorithm/string/replace.hpp>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace fs = std::filesystem;
namespace DB

View File

@ -1,15 +1,7 @@
#include <Common/HTTPHeaderFilter.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
namespace DB
{

View File

@ -484,7 +484,7 @@ OptimizedRegularExpression::OptimizedRegularExpression(const std::string & regex
if (!is_trivial)
{
/// Compile the re2 regular expression.
typename re2::RE2::Options regexp_options;
re2::RE2::Options regexp_options;
/// Never write error messages to stderr. It's ignorant to do it from library code.
regexp_options.set_log_errors(false);

View File

@ -5,17 +5,9 @@
#include <memory>
#include <optional>
#include <Common/StringSearcher.h>
#include <Common/re2.h>
#include "config.h"
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
/** Uses two ways to optimize a regular expression:
* 1. If the regular expression is trivial (reduces to finding a substring in a string),
* then replaces the search with strstr or strcasestr.

View File

@ -3,17 +3,9 @@
#include <Common/RemoteHostFilter.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/Exception.h>
#include <Common/re2.h>
#include <IO/WriteHelpers.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{
namespace ErrorCodes

View File

@ -4,18 +4,10 @@
#include <string>
#include <atomic>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/logger_useful.h>
#include <Common/re2.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>

View File

@ -1,4 +1,5 @@
#include <Common/parseGlobs.h>
#include <Common/re2.h>
#include <IO/WriteBufferFromString.h>
#include <IO/ReadBufferFromString.h>
#include <IO/Operators.h>
@ -6,15 +7,6 @@
#include <sstream>
#include <iomanip>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{
namespace ErrorCodes

11
src/Common/re2.h Normal file
View File

@ -0,0 +1,11 @@
#pragma once
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif

View File

@ -1,15 +1,7 @@
#include <Common/parseGlobs.h>
#include <Common/re2.h>
#include <gtest/gtest.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
using namespace DB;

View File

@ -14,20 +14,12 @@
#include <Storages/HDFS/HDFSCommon.h>
#include <Storages/IStorage.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/re2.h>
#include <Poco/URI.h>
#include <filesystem>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace fs = std::filesystem;
namespace DB

View File

@ -3,6 +3,30 @@
namespace DB
{
String toString(DataSourceType data_source_type)
{
switch (data_source_type)
{
case DataSourceType::Local:
return "local";
case DataSourceType::RAM:
return "memory";
case DataSourceType::S3:
return "s3";
case DataSourceType::S3_Plain:
return "s3_plain";
case DataSourceType::HDFS:
return "hdfs";
case DataSourceType::WebServer:
return "web";
case DataSourceType::AzureBlobStorage:
return "azure_blob_storage";
case DataSourceType::LocalBlobStorage:
return "local_blob_storage";
}
std::unreachable;
}
bool DataSourceDescription::operator==(const DataSourceDescription & other) const
{
return std::tie(type, description, is_encrypted) == std::tie(other.type, other.description, other.is_encrypted);

View File

@ -18,29 +18,7 @@ enum class DataSourceType
LocalBlobStorage,
};
inline String toString(DataSourceType data_source_type)
{
switch (data_source_type)
{
case DataSourceType::Local:
return "local";
case DataSourceType::RAM:
return "memory";
case DataSourceType::S3:
return "s3";
case DataSourceType::S3_Plain:
return "s3_plain";
case DataSourceType::HDFS:
return "hdfs";
case DataSourceType::WebServer:
return "web";
case DataSourceType::AzureBlobStorage:
return "azure_blob_storage";
case DataSourceType::LocalBlobStorage:
return "local_blob_storage";
}
UNREACHABLE();
}
String toString(DataSourceType data_source_type);
struct DataSourceDescription
{

View File

@ -3,19 +3,11 @@
#if USE_AZURE_BLOB_STORAGE
#include <Common/Exception.h>
#include <Common/re2.h>
#include <optional>
#include <azure/identity/managed_identity_credential.hpp>
#include <Poco/Util/AbstractConfiguration.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
using namespace Azure::Storage::Blobs;

View File

@ -5,15 +5,7 @@
#include <Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h>
#include <Disks/ObjectStorages/IMetadataStorage.h>
#include <Disks/ObjectStorages/DiskObjectStorageTransaction.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
namespace CurrentMetrics
{

View File

@ -2,17 +2,9 @@
#include <base/types.h>
#include <Columns/ColumnString.h>
#include <Common/re2.h>
#include <IO/WriteHelpers.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -1,17 +1,9 @@
#pragma once
#include <Common/re2.h>
#include <string_view>
#include <vector>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -13,6 +13,7 @@
#include <Common/logger_useful.h>
#include <Common/Stopwatch.h>
#include <Common/Throttler.h>
#include <Common/re2.h>
#include <IO/HTTPCommon.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
@ -27,14 +28,6 @@
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <boost/algorithm/string.hpp>
static const int SUCCESS_RESPONSE_MIN = 200;

View File

@ -6,18 +6,10 @@
#if USE_AWS_S3
#include <Common/Exception.h>
#include <Common/quoteString.h>
#include <Common/re2.h>
#include <boost/algorithm/string/case_conv.hpp>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -1,17 +1,9 @@
#pragma once
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/OvercommitTracker.h>
#include <Common/re2.h>
#include <Core/Block.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -4,17 +4,9 @@
#include <IO/WriteHelpers.h>
#include <Common/SipHash.h>
#include <Common/quoteString.h>
#include <Common/re2.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -5,17 +5,10 @@
#include <Parsers/ASTIdentifier.h>
#include <Common/SipHash.h>
#include <Common/quoteString.h>
#include <Common/re2.h>
#include <IO/Operators.h>
#include <stack>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -4,8 +4,8 @@
#include <Parsers/Kusto/KustoFunctions/KQLFunctionFactory.h>
#include <format>
#include <regex>
#include <Poco/String.h>
#include <Common/re2.h>
namespace DB
@ -135,12 +135,12 @@ bool ToDecimal::convertImpl(String & out, IParser::Pos & pos)
res = getConvertedArgument(fn_name, pos);
precision = 17;
}
static const std::regex expr{"^[0-9]+e[+-]?[0-9]+"};
bool is_string = std::any_of(res.begin(), res.end(), ::isalpha) && !(std::regex_match(res, expr));
static const re2::RE2 expr("^[0-9]+e[+-]?[0-9]+");
bool is_string = std::any_of(res.begin(), res.end(), ::isalpha) && !(re2::RE2::FullMatch(res, expr));
if (is_string)
out = "NULL";
else if (std::regex_match(res, expr))
else if (re2::RE2::FullMatch(res, expr))
{
auto exponential_pos = res.find('e');
if (res[exponential_pos + 1] == '+' || res[exponential_pos + 1] == '-')

View File

@ -1,3 +1,4 @@
#include <Common/re2.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/IParserBase.h>
@ -9,7 +10,6 @@
#include <Parsers/ParserSetQuery.h>
#include "Poco/String.h"
#include <format>
#include <regex>
namespace DB
{
@ -224,13 +224,14 @@ bool DatatypeDecimal::convertImpl(String & out, IParser::Pos & pos)
--pos;
arg = getArgument(fn_name, pos);
//NULL expr returns NULL not exception
static const std::regex expr{"^[0-9]+e[+-]?[0-9]+"};
bool is_string = std::any_of(arg.begin(), arg.end(), ::isalpha) && Poco::toUpper(arg) != "NULL" && !(std::regex_match(arg, expr));
/// NULL expr returns NULL not exception
static const re2::RE2 expr("^[0-9]+e[+-]?[0-9]+");
assert(expr.ok());
bool is_string = std::any_of(arg.begin(), arg.end(), ::isalpha) && Poco::toUpper(arg) != "NULL" && !(re2::RE2::FullMatch(arg, expr));
if (is_string)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to parse String as decimal Literal: {}", fn_name);
if (std::regex_match(arg, expr))
if (re2::RE2::FullMatch(arg, expr))
{
auto exponential_pos = arg.find('e');
if (arg[exponential_pos + 1] == '+' || arg[exponential_pos + 1] == '-')

View File

@ -1,5 +1,4 @@
#include <format>
#include <regex>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/IParserBase.h>

View File

@ -15,12 +15,13 @@
#include <Parsers/parseQuery.h>
#include <Parsers/Kusto/ParserKQLQuery.h>
#include <Parsers/PRQL/ParserPRQLQuery.h>
#include <Common/re2.h>
#include <string_view>
#include <regex>
#include <gtest/gtest.h>
#include "gtest_common.h"
#include <boost/algorithm/string/replace.hpp>
namespace
{
using namespace DB;
@ -71,14 +72,14 @@ TEST_P(ParserTest, parseQuery)
if (input_text.starts_with("ATTACH"))
{
auto salt = (dynamic_cast<const ASTCreateUserQuery *>(ast.get())->auth_data)->getSalt().value_or("");
EXPECT_TRUE(std::regex_match(salt, std::regex(expected_ast)));
EXPECT_TRUE(re2::RE2::FullMatch(salt, expected_ast));
}
else
{
WriteBufferFromOwnString buf;
formatAST(*ast->clone(), buf, false, false);
String formatted_ast = buf.str();
EXPECT_TRUE(std::regex_match(formatted_ast, std::regex(expected_ast)));
EXPECT_TRUE(re2::RE2::FullMatch(formatted_ast, expected_ast));
}
}
}

View File

@ -6,9 +6,11 @@
#include <Parsers/parseQuery.h>
#include <Parsers/Kusto/parseKQLQuery.h>
#include <Common/re2.h>
#include <gmock/gmock.h>
#include <regex>
namespace
{
@ -62,14 +64,14 @@ TEST_P(ParserKQLTest, parseKQLQuery)
if (input_text.starts_with("ATTACH"))
{
auto salt = (dynamic_cast<const ASTCreateUserQuery *>(ast.get())->auth_data)->getSalt().value_or("");
EXPECT_TRUE(std::regex_match(salt, std::regex(expected_ast)));
EXPECT_TRUE(re2::RE2::FullMatch(salt, expected_ast));
}
else
{
DB::WriteBufferFromOwnString buf;
formatAST(*ast->clone(), buf, false, false);
String formatted_ast = buf.str();
EXPECT_TRUE(std::regex_match(formatted_ast, std::regex(expected_ast)));
EXPECT_TRUE(re2::RE2::FullMatch(formatted_ast, expected_ast));
}
}
}

View File

@ -27,6 +27,8 @@
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnMap.h>
#include <Common/re2.h>
#include <DataFile.hh>
#include <Encoder.hh>
#include <Node.hh>
@ -34,15 +36,6 @@
#include <boost/algorithm/string.hpp>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{
namespace ErrorCodes

View File

@ -3,6 +3,7 @@
#include <string>
#include <vector>
#include <Core/Block.h>
#include <Common/re2.h>
#include <IO/PeekableReadBuffer.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
@ -11,15 +12,6 @@
#include <Formats/ParsedTemplateFormatString.h>
#include <Formats/SchemaInferenceUtils.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -27,6 +27,7 @@
#include <Common/scope_guard_safe.h>
#include <Common/setThreadName.h>
#include <Common/typeid_cast.h>
#include <Common/re2.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatFactory.h>
@ -49,15 +50,6 @@
#include <chrono>
#include <sstream>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#if USE_SSL
#include <Poco/Net/X509Certificate.h>
#endif

View File

@ -6,15 +6,7 @@
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
#include <Common/CurrentMetrics.h>
#include <Common/CurrentThread.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
namespace CurrentMetrics
{

View File

@ -4,21 +4,13 @@
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <base/find_symbols.h>
#include <Common/re2.h>
#include <Poco/StringTokenizer.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <unordered_map>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -2,7 +2,6 @@
#include <limits>
#include <optional>
#include <regex>
#include <Core/MySQL/Authentication.h>
#include <Core/MySQL/PacketsConnection.h>
#include <Core/MySQL/PacketsGeneric.h>
@ -26,6 +25,7 @@
#include <Common/logger_useful.h>
#include <Common/setThreadName.h>
#include <Common/config_version.h>
#include <Common/re2.h>
#if USE_SSL
# include <Poco/Crypto/RSAKey.h>
@ -524,16 +524,18 @@ void MySQLHandlerSSL::finishHandshakeSSL(
static bool isFederatedServerSetupSetCommand(const String & query)
{
static const std::regex expr{
re2::RE2::Options regexp_options;
regexp_options.set_case_sensitive(false);
static const re2::RE2 expr(
"(^(SET NAMES(.*)))"
"|(^(SET character_set_results(.*)))"
"|(^(SET FOREIGN_KEY_CHECKS(.*)))"
"|(^(SET AUTOCOMMIT(.*)))"
"|(^(SET sql_mode(.*)))"
"|(^(SET @@(.*)))"
"|(^(SET SESSION TRANSACTION ISOLATION LEVEL(.*)))"
, std::regex::icase};
return 1 == std::regex_match(query, expr);
"|(^(SET SESSION TRANSACTION ISOLATION LEVEL(.*)))", regexp_options);
assert(expr.ok());
return re2::RE2::PartialMatch(query, expr);
}
/// Replace "[query(such as SHOW VARIABLES...)]" into "".
@ -592,8 +594,8 @@ static String killConnectionIdReplacementQuery(const String & query)
if (query.size() > prefix.size())
{
String suffix = query.data() + prefix.length();
static const std::regex expr{"^[0-9]"};
if (std::regex_match(suffix, expr))
static const re2::RE2 expr("^[0-9]");
if (re2::RE2::FullMatch(suffix, expr))
{
String replacement = fmt::format("KILL QUERY WHERE query_id LIKE 'mysql:{}:%'", suffix);
return replacement;

View File

@ -1,10 +1,9 @@
#include "PrometheusMetricsWriter.h"
#include <IO/WriteHelpers.h>
#include <regex> /// TODO: this library is harmful.
#include <Common/re2.h>
#include <algorithm>
namespace
{
@ -26,9 +25,11 @@ void writeOutLine(DB::WriteBuffer & wb, T && val, TArgs &&... args)
/// Returns false if name is not valid
bool replaceInvalidChars(std::string & metric_name)
{
/// dirty solution
metric_name = std::regex_replace(metric_name, std::regex("[^a-zA-Z0-9_:]"), "_");
metric_name = std::regex_replace(metric_name, std::regex("^[^a-zA-Z]*"), "");
/// dirty solution:
static const re2::RE2 regexp1("[^a-zA-Z0-9_:]");
static const re2::RE2 regexp2("^[^a-zA-Z]*");
re2::RE2::GlobalReplace(&metric_name, regexp1, "_");
re2::RE2::GlobalReplace(&metric_name, regexp2, "");
return !metric_name.empty();
}

View File

@ -44,8 +44,8 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe
}
}
HTTPRequestHandlerFactoryPtr
createPrometheusHandlerFactory(IServer & server,
HTTPRequestHandlerFactoryPtr createPrometheusHandlerFactory(
IServer & server,
const Poco::Util::AbstractConfiguration & config,
AsynchronousMetrics & async_metrics,
const std::string & config_prefix)

View File

@ -16,7 +16,7 @@ private:
const PrometheusMetricsWriter & metrics_writer;
public:
explicit PrometheusRequestHandler(IServer & server_, const PrometheusMetricsWriter & metrics_writer_)
PrometheusRequestHandler(IServer & server_, const PrometheusMetricsWriter & metrics_writer_)
: server(server_)
, metrics_writer(metrics_writer_)
{

View File

@ -6,15 +6,7 @@
#include <Interpreters/Context.h>
#include <IO/HTTPCommon.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
#include <incbin.h>

View File

@ -2,15 +2,7 @@
#include <Poco/URI.h>
#include <boost/algorithm/string/replace.hpp>
#include <filesystem>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
#include <Common/re2.h>
#if USE_HDFS
#include <Common/ShellCommand.h>

View File

@ -3,6 +3,7 @@
#if USE_HDFS
#include <Common/parseGlobs.h>
#include <Common/re2.h>
#include <DataTypes/DataTypeString.h>
#include <Parsers/ASTLiteral.h>
@ -48,15 +49,6 @@
#include <filesystem>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace fs = std::filesystem;
namespace ProfileEvents

View File

@ -22,7 +22,6 @@
#include <Poco/Net/HTTPRequest.h>
#include <boost/algorithm/string/join.hpp>
#include <iterator>
#include <regex>
#include <base/sort.h>
@ -169,10 +168,19 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
String remote_fs_metadata = parse<String>(params.get("remote_fs_metadata", ""));
std::regex re("\\s*,\\s*");
Strings capability(
std::sregex_token_iterator(remote_fs_metadata.begin(), remote_fs_metadata.end(), re, -1),
std::sregex_token_iterator());
/// Tokenize capabilities from remote_fs_metadata
/// E.g. remote_fs_metadata = "local, s3_plain, web" --> capabilities = ["local", "s3_plain", "web"]
Strings capabilities;
const String delimiter(", ");
size_t pos_start = 0;
size_t pos_end;
while ((pos_end = remote_fs_metadata.find(delimiter, pos_start)) != std::string::npos)
{
const String token = remote_fs_metadata.substr(pos_start, pos_end - pos_start);
pos_start = pos_end + delimiter.size();
capabilities.push_back(token);
}
capabilities.push_back(remote_fs_metadata.substr(pos_start));
bool send_projections = client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION;
@ -188,9 +196,9 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write
client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_ZERO_COPY)
{
auto disk_type = part->getDataPartStorage().getDiskType();
if (part->getDataPartStorage().supportZeroCopyReplication() && std::find(capability.begin(), capability.end(), disk_type) != capability.end())
if (part->getDataPartStorage().supportZeroCopyReplication() && std::find(capabilities.begin(), capabilities.end(), disk_type) != capabilities.end())
{
/// Send metadata if the receiver's capability covers the source disk type.
/// Send metadata if the receiver's capabilities covers the source disk type.
response.addCookie({"remote_fs_metadata", disk_type});
sendPartFromDisk(part, out, client_protocol_version, true, send_projections);
return;

View File

@ -3,10 +3,11 @@
#include <IO/HTTPHeaderEntries.h>
#include <Common/NamedCollections/NamedCollections.h>
#include <Common/quoteString.h>
#include <Common/re2.h>
#include <unordered_set>
#include <string_view>
#include <fmt/format.h>
#include <regex>
namespace ErrorCodes
{
@ -96,7 +97,7 @@ void validateNamedCollection(
const NamedCollection & collection,
const Keys & required_keys,
const Keys & optional_keys,
const std::vector<std::regex> & optional_regex_keys = {})
const std::vector<std::shared_ptr<re2::RE2>> & optional_regex_keys = {})
{
NamedCollection::Keys keys = collection.getKeys();
auto required_keys_copy = required_keys;
@ -119,7 +120,7 @@ void validateNamedCollection(
auto match = std::find_if(
optional_regex_keys.begin(), optional_regex_keys.end(),
[&](const std::regex & regex) { return std::regex_search(key, regex); })
[&](const std::shared_ptr<re2::RE2> & regex) { return re2::RE2::PartialMatch(key, *regex); })
!= optional_regex_keys.end();
if (!match)

View File

@ -6,6 +6,7 @@
#include <Storages/checkAndGetLiteralArgument.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTInsertQuery.h>
#include <Common/re2.h>
#include <IO/SharedThreadPools.h>
@ -40,15 +41,6 @@
#include <Disks/IO/ReadBufferFromAzureBlobStorage.h>
#include <Disks/IO/WriteBufferFromAzureBlobStorage.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
using namespace Azure::Storage::Blobs;
namespace CurrentMetrics

View File

@ -4,6 +4,7 @@
#if USE_AZURE_BLOB_STORAGE
#include <Common/re2.h>
#include <Storages/IStorage.h>
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
#include <Storages/Cache/SchemaCache.h>
@ -14,15 +15,6 @@
#include <Storages/prepareReadingFromFormat.h>
#include <Storages/SelectQueryInfo.h>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -47,6 +47,7 @@
#include <Common/filesystemHelpers.h>
#include <Common/logger_useful.h>
#include <Common/ProfileEvents.h>
#include <Common/re2.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
@ -57,15 +58,6 @@
#include <shared_mutex>
#include <algorithm>
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace ProfileEvents
{
extern const Event CreatedReadBufferOrdinary;

View File

@ -36,13 +36,13 @@
#include <Common/ProfileEvents.h>
#include <Common/thread_local_rng.h>
#include <Common/logger_useful.h>
#include <Common/re2.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/HTTPHeaderEntries.h>
#include <algorithm>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Poco/Net/HTTPRequest.h>
#include <regex>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeLowCardinality.h>
@ -84,9 +84,9 @@ static const std::unordered_set<std::string_view> optional_configuration_keys =
/// Headers in config file will have structure "headers.header.name" and "headers.header.value".
/// But Poco::AbstractConfiguration converts them into "header", "header[1]", "header[2]".
static const std::vector<std::regex> optional_regex_keys = {
std::regex(R"(headers.header\[[\d]*\].name)"),
std::regex(R"(headers.header\[[\d]*\].value)"),
static const std::vector<std::shared_ptr<re2::RE2>> optional_regex_keys = {
std::make_shared<re2::RE2>(R"(headers.header\[[0-9]*\].name)"),
std::make_shared<re2::RE2>(R"(headers.header\[[0-9]*\].value)"),
};
static bool urlWithGlobs(const String & uri)

View File

@ -1,4 +1,5 @@
#include "config.h"
#include <Common/re2.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
@ -13,15 +14,6 @@
#include "Poco/Crypto/X509Certificate.h"
#endif
#ifdef __clang__
# pragma clang diagnostic push
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
#include <re2/re2.h>
#ifdef __clang__
# pragma clang diagnostic pop
#endif
namespace DB
{

View File

@ -1,5 +1,4 @@
#include <numeric>
#include <regex>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
@ -54,6 +53,7 @@
#include <Common/ProfileEvents.h>
#include <base/sleep.h>
#include <Common/logger_useful.h>
#include <boost/algorithm/string/replace.hpp>
#include <Storages/LiveView/StorageBlocks.h>
@ -63,6 +63,7 @@
#include <QueryPipeline/printPipeline.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
namespace DB
{
namespace ErrorCodes
@ -1268,7 +1269,7 @@ ASTPtr StorageWindowView::initInnerQuery(ASTSelectQuery query, ContextPtr contex
if (is_time_column_func_now)
window_id_name = func_now_data.window_id_name;
window_column_name = std::regex_replace(window_id_name, std::regex("windowID"), is_tumble ? "tumble" : "hop");
window_column_name = boost::replace_all_copy(window_id_name, "windowID", is_tumble ? "tumble" : "hop");
/// Parse final query (same as mergeable query but has tumble/hop instead of windowID)
final_query = mergeable_query->clone();

View File

@ -420,6 +420,9 @@ find $ROOT_PATH/tests/queries/1_stateful -name '*.sql' -or -name '*.sh' | grep -
# Check for bad punctuation: whitespace before comma.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '\w ,' | grep -v 'bad punctuation is ok here' && echo "^ There is bad punctuation: whitespace before comma. You should write it like this: 'Hello, world!'"
# Check usage of std::regex which is too bloated and slow.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number 'std::regex' | grep -P '.' && echo "^ Please use re2 instead of std::regex"
# Cyrillic characters hiding inside Latin.
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | xargs grep -P --line-number '[a-zA-Z][а-яА-ЯёЁ]|[а-яА-ЯёЁ][a-zA-Z]' && echo "^ Cyrillic characters found in unexpected place."