Remove broken lockless variant of re2

This commit is contained in:
Robert Schulze 2023-09-14 16:12:29 +00:00
parent 49f29861a0
commit 7b378dbad3
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
54 changed files with 375 additions and 179 deletions

View File

@ -1,14 +1,3 @@
# Copyright 2015 The RE2 Authors. All Rights Reserved.
# Use of this source code is governed by a BSD-style
# license that can be found in the LICENSE file.
# This file was edited for ClickHouse
string(FIND ${CMAKE_CURRENT_BINARY_DIR} " " _have_space)
if(_have_space GREATER 0)
message(FATAL_ERROR "Using spaces in build path [${CMAKE_CURRENT_BINARY_DIR}] highly not recommended. Library re2st will be disabled.")
endif()
set(SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/re2")
set(RE2_SOURCES
@ -35,33 +24,9 @@ set(RE2_SOURCES
${SRC_DIR}/util/rune.cc
${SRC_DIR}/util/strutil.cc
)
add_library(re2 ${RE2_SOURCES})
target_include_directories(re2 PUBLIC "${SRC_DIR}")
target_link_libraries(re2 ch_contrib::abseil_str_format)
# Building re2 which is thread-safe and re2_st which is not.
# re2 changes its state during matching of regular expression, e.g. creates temporary DFA.
# It uses RWLock to process the same regular expression object from different threads.
# In order to avoid redundant locks in some cases, we use not thread-safe version of the library (re2_st).
add_library(_re2 ${RE2_SOURCES})
target_include_directories(_re2 PUBLIC "${SRC_DIR}")
target_link_libraries(_re2 ch_contrib::abseil_str_format)
add_library(re2_st ${RE2_SOURCES})
target_compile_definitions (re2_st PRIVATE NDEBUG NO_THREADS re2=re2_st)
target_include_directories (re2_st PRIVATE .)
target_include_directories (re2_st SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR})
target_include_directories (re2_st SYSTEM BEFORE PUBLIC ${SRC_DIR})
target_link_libraries (re2_st ch_contrib::abseil_str_format)
file (MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/re2_st)
foreach (FILENAME filtered_re2.h re2.h set.h stringpiece.h)
add_custom_command (OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/re2_st/${FILENAME}"
COMMAND ${CMAKE_COMMAND} -DSOURCE_FILENAME="${SRC_DIR}/re2/${FILENAME}"
-DTARGET_FILENAME="${CMAKE_CURRENT_BINARY_DIR}/re2_st/${FILENAME}"
-P "${CMAKE_CURRENT_SOURCE_DIR}/re2_transform.cmake"
COMMENT "Creating ${FILENAME} for re2_st library.")
add_custom_target (transform_${FILENAME} DEPENDS "${CMAKE_CURRENT_BINARY_DIR}/re2_st/${FILENAME}")
add_dependencies (re2_st transform_${FILENAME})
endforeach ()
# NOTE: you should not change name of library here, since it is used to generate required header (see above)
add_library(ch_contrib::re2 ALIAS re2)
add_library(ch_contrib::re2_st ALIAS re2_st)
add_library(ch_contrib::re2 ALIAS _re2)

View File

@ -1,10 +0,0 @@
file (READ ${SOURCE_FILENAME} CONTENT)
string (REGEX REPLACE "using re2::RE2;" "" CONTENT "${CONTENT}")
string (REGEX REPLACE "using re2::LazyRE2;" "" CONTENT "${CONTENT}")
string (REGEX REPLACE "namespace re2 {" "namespace re2_st {" CONTENT "${CONTENT}")
string (REGEX REPLACE "re2::" "re2_st::" CONTENT "${CONTENT}")
string (REGEX REPLACE "\"re2/" "\"re2_st/" CONTENT "${CONTENT}")
string (REGEX REPLACE "(.\\*?_H)" "\\1_ST" CONTENT "${CONTENT}")
string (REGEX REPLACE "#define MUTEX_IS_PTHREAD_RWLOCK" "#undef MUTEX_IS_PTHREAD_RWLOCK" CONTENT "${CONTENT}")
string (REGEX REPLACE "typedef std::mutex MutexType;" "struct MutexType { void lock() {} void unlock() {} };" CONTENT "${CONTENT}")
file (WRITE ${TARGET_FILENAME} "${CONTENT}")

View File

@ -18,7 +18,14 @@
#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
static void setupLogging(const std::string & log_level)
{

View File

@ -9,8 +9,6 @@
#include <thread>
#include <filesystem>
#include <re2/re2.h>
#include <boost/program_options.hpp>
#include <Common/TerminalSize.h>
@ -26,6 +24,14 @@
#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

@ -10,9 +10,17 @@
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <boost/program_options.hpp>
#include <re2/re2.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;
#define EXTRACT_PATH_PATTERN ".*\\/store/(.*)"

View File

@ -26,10 +26,17 @@
#include <IO/Operators.h>
#include <Poco/AccessExpireCache.h>
#include <boost/algorithm/string/join.hpp>
#include <re2/re2.h>
#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
{

View File

@ -1,11 +1,18 @@
#pragma once
#include <re2/re2.h>
#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
namespace DB
{

View File

@ -1,12 +1,18 @@
#pragma once
#include <re2/re2.h>
#include <Analyzer/Identifier.h>
#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
namespace DB
{

View File

@ -316,7 +316,6 @@ target_link_libraries(clickhouse_common_io
boost::context
ch_contrib::cityhash
ch_contrib::re2
ch_contrib::re2_st
ch_contrib::zlib
pcg_random
Poco::Foundation

View File

@ -7,11 +7,19 @@
#include <filesystem>
#include <format>
#include <map>
#include <re2/re2.h>
#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

@ -2,7 +2,14 @@
#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
namespace DB
{

View File

@ -441,8 +441,7 @@ finish:
}
}
template <bool thread_safe>
void OptimizedRegularExpressionImpl<thread_safe>::analyze(
void OptimizedRegularExpression::analyze(
std::string_view regexp_,
std::string & required_substring,
bool & is_trivial,
@ -467,8 +466,7 @@ catch (...)
LOG_ERROR(&Poco::Logger::get("OptimizeRegularExpression"), "Analyze RegularExpression failed, got error: {}", DB::getCurrentExceptionMessage(false));
}
template <bool thread_safe>
OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(const std::string & regexp_, int options)
OptimizedRegularExpression::OptimizedRegularExpression(const std::string & regexp_, int options)
{
std::vector<std::string> alternatives_dummy; /// this vector extracts patterns a,b,c from pattern (a|b|c). for now it's not used.
analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix, alternatives_dummy);
@ -486,7 +484,7 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
if (!is_trivial)
{
/// Compile the re2 regular expression.
typename RegexType::Options regexp_options;
typename 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);
@ -497,7 +495,7 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
if (is_dot_nl)
regexp_options.set_dot_nl(true);
re2 = std::make_unique<RegexType>(regexp_, regexp_options);
re2 = std::make_unique<re2::RE2>(regexp_, regexp_options);
if (!re2->ok())
{
throw DB::Exception(DB::ErrorCodes::CANNOT_COMPILE_REGEXP,
@ -527,8 +525,7 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(cons
}
}
template <bool thread_safe>
OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(OptimizedRegularExpressionImpl && rhs) noexcept
OptimizedRegularExpression::OptimizedRegularExpression(OptimizedRegularExpression && rhs) noexcept
: is_trivial(rhs.is_trivial)
, required_substring_is_prefix(rhs.required_substring_is_prefix)
, is_case_insensitive(rhs.is_case_insensitive)
@ -545,8 +542,7 @@ OptimizedRegularExpressionImpl<thread_safe>::OptimizedRegularExpressionImpl(Opti
}
}
template <bool thread_safe>
bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, size_t subject_size) const
bool OptimizedRegularExpression::match(const char * subject, size_t subject_size) const
{
const UInt8 * haystack = reinterpret_cast<const UInt8 *>(subject);
const UInt8 * haystack_end = haystack + subject_size;
@ -577,13 +573,12 @@ bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, si
}
}
return re2->Match({subject, subject_size}, 0, subject_size, RegexType::UNANCHORED, nullptr, 0);
return re2->Match({subject, subject_size}, 0, subject_size, re2::RE2::UNANCHORED, nullptr, 0);
}
}
template <bool thread_safe>
bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, size_t subject_size, Match & match) const
bool OptimizedRegularExpression::match(const char * subject, size_t subject_size, Match & match) const
{
const UInt8 * haystack = reinterpret_cast<const UInt8 *>(subject);
const UInt8 * haystack_end = haystack + subject_size;
@ -624,7 +619,7 @@ bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, si
std::string_view piece;
if (!RegexType::PartialMatch({subject, subject_size}, *re2, &piece))
if (!re2::RE2::PartialMatch({subject, subject_size}, *re2, &piece))
return false;
else
{
@ -636,8 +631,7 @@ bool OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, si
}
template <bool thread_safe>
unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject, size_t subject_size, MatchVec & matches, unsigned limit) const
unsigned OptimizedRegularExpression::match(const char * subject, size_t subject_size, MatchVec & matches, unsigned limit) const
{
const UInt8 * haystack = reinterpret_cast<const UInt8 *>(subject);
const UInt8 * haystack_end = haystack + subject_size;
@ -695,7 +689,7 @@ unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject
{subject, subject_size},
0,
subject_size,
RegexType::UNANCHORED,
re2::RE2::UNANCHORED,
pieces.data(),
static_cast<int>(pieces.size())))
{
@ -721,6 +715,3 @@ unsigned OptimizedRegularExpressionImpl<thread_safe>::match(const char * subject
}
}
}
template class OptimizedRegularExpressionImpl<true>;
template class OptimizedRegularExpressionImpl<false>;

View File

@ -6,9 +6,15 @@
#include <optional>
#include <Common/StringSearcher.h>
#include "config.h"
#include <re2/re2.h>
#include <re2_st/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
/** Uses two ways to optimize a regular expression:
* 1. If the regular expression is trivial (reduces to finding a substring in a string),
@ -37,8 +43,7 @@ namespace OptimizedRegularExpressionDetails
};
}
template <bool thread_safe>
class OptimizedRegularExpressionImpl
class OptimizedRegularExpression
{
public:
enum Options
@ -51,12 +56,10 @@ public:
using Match = OptimizedRegularExpressionDetails::Match;
using MatchVec = std::vector<Match>;
using RegexType = std::conditional_t<thread_safe, re2::RE2, re2_st::RE2>;
OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0); /// NOLINT
OptimizedRegularExpression(const std::string & regexp_, int options = 0); /// NOLINT
/// StringSearcher store pointers to required_substring, it must be updated on move.
OptimizedRegularExpressionImpl(OptimizedRegularExpressionImpl && rhs) noexcept;
OptimizedRegularExpressionImpl(const OptimizedRegularExpressionImpl & rhs) = delete;
OptimizedRegularExpression(OptimizedRegularExpression && rhs) noexcept;
OptimizedRegularExpression(const OptimizedRegularExpression & rhs) = delete;
bool match(const std::string & subject) const
{
@ -85,7 +88,7 @@ public:
unsigned getNumberOfSubpatterns() const { return number_of_subpatterns; }
/// Get the regexp re2 or nullptr if the pattern is trivial (for output to the log).
const std::unique_ptr<RegexType> & getRE2() const { return re2; }
const std::unique_ptr<re2::RE2> & getRE2() const { return re2; }
void getAnalyzeResult(std::string & out_required_substring, bool & out_is_trivial, bool & out_required_substring_is_prefix) const
{
@ -110,9 +113,6 @@ private:
std::string required_substring;
std::optional<DB::ASCIICaseSensitiveStringSearcher> case_sensitive_substring_searcher;
std::optional<DB::ASCIICaseInsensitiveStringSearcher> case_insensitive_substring_searcher;
std::unique_ptr<RegexType> re2;
std::unique_ptr<re2::RE2> re2;
unsigned number_of_subpatterns;
};
using OptimizedRegularExpression = OptimizedRegularExpressionImpl<true>;
using OptimizedRegularExpressionSingleThreaded = OptimizedRegularExpressionImpl<false>;

View File

@ -1,4 +1,3 @@
#include <re2/re2.h>
#include <Poco/URI.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/RemoteHostFilter.h>
@ -6,6 +5,14 @@
#include <Common/Exception.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

@ -4,7 +4,14 @@
#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>

View File

@ -2,11 +2,18 @@
#include <IO/WriteBufferFromString.h>
#include <IO/ReadBufferFromString.h>
#include <IO/Operators.h>
#include <re2/re2.h>
#include <algorithm>
#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
{

View File

@ -1,7 +1,14 @@
#include <Common/parseGlobs.h>
#include <re2/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

@ -15,10 +15,18 @@
#include <TableFunctions/TableFunctionFactory.h>
#include <Poco/URI.h>
#include <re2/re2.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

@ -117,14 +117,14 @@ struct RegExpTreeDictionary::RegexTreeNode
UInt64 id;
UInt64 parent_id;
std::string regex;
re2_st::RE2 searcher;
re2::RE2 searcher;
RegexTreeNode(UInt64 id_, UInt64 parent_id_, const String & regex_, const re2_st::RE2::Options & regexp_options):
RegexTreeNode(UInt64 id_, UInt64 parent_id_, const String & regex_, const re2::RE2::Options & regexp_options):
id(id_), parent_id(parent_id_), regex(regex_), searcher(regex_, regexp_options) {}
bool match(const char * haystack, size_t size) const
{
return searcher.Match(haystack, 0, size, re2_st::RE2::Anchor::UNANCHORED, nullptr, 0);
return searcher.Match(haystack, 0, size, re2::RE2::Anchor::UNANCHORED, nullptr, 0);
}
struct AttributeValue
@ -204,7 +204,7 @@ void RegExpTreeDictionary::initRegexNodes(Block & block)
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "There are invalid id {}", id);
re2_st::RE2::Options regexp_options;
re2::RE2::Options regexp_options;
regexp_options.set_log_errors(false);
regexp_options.set_case_sensitive(!flag_case_insensitive);
regexp_options.set_dot_nl(flag_dotall);
@ -480,11 +480,11 @@ public:
inline size_t attributesFull() const { return n_full_attributes; }
};
std::pair<String, bool> processBackRefs(const String & data, const re2_st::RE2 & searcher, const std::vector<StringPiece> & pieces)
std::pair<String, bool> processBackRefs(const String & data, const re2::RE2 & searcher, const std::vector<StringPiece> & pieces)
{
std::string_view matches[10];
String result;
searcher.Match({data.data(), data.size()}, 0, data.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 10);
searcher.Match({data.data(), data.size()}, 0, data.size(), re2::RE2::Anchor::UNANCHORED, matches, 10);
/// if the pattern is a single '$1' but fails to match, we would use the default value.
if (pieces.size() == 1 && pieces[0].ref_num >= 0 && pieces[0].ref_num < 10 && matches[pieces[0].ref_num].empty())
return std::make_pair(result, true);

View File

@ -4,10 +4,18 @@
#include <Common/Exception.h>
#include <optional>
#include <re2/re2.h>
#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,7 +5,15 @@
#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
namespace CurrentMetrics
{

View File

@ -464,7 +464,7 @@ public:
"Must be constant string.", arguments[0].column->getName(), getName());
if (!col->getValue<String>().empty())
re = std::make_shared<Regexps::Regexp>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
re = std::make_shared<OptimizedRegularExpression>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
}
/// Returns the position of the argument that is the column of strings
@ -551,7 +551,7 @@ public:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. "
"Must be constant string.", arguments[1].column->getName(), getName());
re = std::make_shared<Regexps::Regexp>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
re = std::make_shared<OptimizedRegularExpression>(Regexps::createRegexp<false, false, false>(col->getValue<String>()));
capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0;
matches.resize(capture + 1);

View File

@ -9,7 +9,7 @@
#include <Functions/Regexps.h>
#include "config.h"
#include <re2_st/re2.h>
#include <re2/re2.h>
namespace DB
@ -191,7 +191,7 @@ struct MatchImpl
return;
}
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
const auto & regexp = OptimizedRegularExpression(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
String required_substring;
bool is_trivial;
@ -212,7 +212,7 @@ struct MatchImpl
{reinterpret_cast<const char *>(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1},
0,
haystack_offsets[i] - prev_offset - 1,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
@ -266,7 +266,7 @@ struct MatchImpl
{str_data, str_size},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
@ -359,7 +359,7 @@ struct MatchImpl
return;
}
const auto & regexp = Regexps::Regexp(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
const auto & regexp = OptimizedRegularExpression(Regexps::createRegexp<is_like, /*no_capture*/ true, case_insensitive>(needle));
String required_substring;
bool is_trivial;
@ -380,7 +380,7 @@ struct MatchImpl
{reinterpret_cast<const char *>(&haystack[offset]), N},
0,
N,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
@ -437,7 +437,7 @@ struct MatchImpl
{str_data, N},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
@ -526,7 +526,7 @@ struct MatchImpl
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
0,
cur_haystack_length,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
@ -552,7 +552,7 @@ struct MatchImpl
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match2;
@ -635,7 +635,7 @@ struct MatchImpl
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
0,
cur_haystack_length,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match;
@ -661,7 +661,7 @@ struct MatchImpl
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
res[i] = negate ^ match2;

View File

@ -341,7 +341,7 @@ struct MultiMatchAnyImpl
{
String needle(needles[j]);
const auto & regexp = Regexps::Regexp(Regexps::createRegexp</*like*/ false, /*no_capture*/ true, /*case_insensitive*/ false>(needle));
const auto & regexp = OptimizedRegularExpression(Regexps::createRegexp</*like*/ false, /*no_capture*/ true, /*case_insensitive*/ false>(needle));
String required_substr;
bool is_trivial;
@ -364,7 +364,7 @@ struct MultiMatchAnyImpl
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
0,
cur_haystack_length,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
if constexpr (FindAny)
@ -401,7 +401,7 @@ struct MultiMatchAnyImpl
{reinterpret_cast<const char *>(cur_haystack_data), cur_haystack_length},
start_pos,
end_pos,
re2_st::RE2::UNANCHORED,
re2::RE2::UNANCHORED,
nullptr,
0);
if constexpr (FindAny)

View File

@ -39,11 +39,10 @@ namespace ErrorCodes
namespace Regexps
{
using Regexp = OptimizedRegularExpressionSingleThreaded;
using RegexpPtr = std::shared_ptr<Regexp>;
using RegexpPtr = std::shared_ptr<OptimizedRegularExpression>;
template <bool like, bool no_capture, bool case_insensitive>
inline Regexp createRegexp(const String & pattern)
inline OptimizedRegularExpression createRegexp(const String & pattern)
{
int flags = OptimizedRegularExpression::RE_DOT_NL;
if constexpr (no_capture)
@ -65,7 +64,7 @@ inline Regexp createRegexp(const String & pattern)
class LocalCacheTable
{
public:
using RegexpPtr = std::shared_ptr<Regexp>;
using RegexpPtr = std::shared_ptr<OptimizedRegularExpression>;
template <bool like, bool no_capture, bool case_insensitive>
RegexpPtr getOrSet(const String & pattern)
@ -74,11 +73,11 @@ public:
if (bucket.regexp == nullptr) [[unlikely]]
/// insert new entry
bucket = {pattern, std::make_shared<Regexp>(createRegexp<like, no_capture, case_insensitive>(pattern))};
bucket = {pattern, std::make_shared<OptimizedRegularExpression>(createRegexp<like, no_capture, case_insensitive>(pattern))};
else
if (pattern != bucket.pattern)
/// replace existing entry
bucket = {pattern, std::make_shared<Regexp>(createRegexp<like, no_capture, case_insensitive>(pattern))};
bucket = {pattern, std::make_shared<OptimizedRegularExpression>(createRegexp<like, no_capture, case_insensitive>(pattern))};
return bucket.regexp;
}

View File

@ -4,9 +4,14 @@
#include <Columns/ColumnString.h>
#include <IO/WriteHelpers.h>
#include "config.h"
#include <re2_st/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
{
@ -95,7 +100,7 @@ struct ReplaceRegexpImpl
size_t haystack_length,
ColumnString::Chars & res_data,
ColumnString::Offset & res_offset,
const re2_st::RE2 & searcher,
const re2::RE2 & searcher,
int num_captures,
const Instructions & instructions)
{
@ -110,7 +115,7 @@ struct ReplaceRegexpImpl
/// If no more replacements possible for current string
bool can_finish_current_string = false;
if (searcher.Match(haystack, match_pos, haystack_length, re2_st::RE2::Anchor::UNANCHORED, matches, num_captures))
if (searcher.Match(haystack, match_pos, haystack_length, re2::RE2::Anchor::UNANCHORED, matches, num_captures))
{
const auto & match = matches[0]; /// Complete match (\0)
size_t bytes_to_copy = (match.data() - haystack.data()) - copy_pos;
@ -181,11 +186,11 @@ struct ReplaceRegexpImpl
size_t haystack_size = haystack_offsets.size();
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
re2::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
re2_st::RE2 searcher(needle, regexp_options);
re2::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
@ -223,7 +228,7 @@ struct ReplaceRegexpImpl
size_t haystack_size = haystack_offsets.size();
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
re2::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
@ -242,7 +247,7 @@ struct ReplaceRegexpImpl
if (needle.empty())
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Length of the pattern argument in function {} must be greater than 0.", name);
re2_st::RE2 searcher(needle, regexp_options);
re2::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures);
@ -272,11 +277,11 @@ struct ReplaceRegexpImpl
size_t haystack_size = haystack_offsets.size();
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
re2::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
re2_st::RE2 searcher(needle, regexp_options);
re2::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
@ -319,7 +324,7 @@ struct ReplaceRegexpImpl
size_t haystack_size = haystack_offsets.size();
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
re2::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
@ -342,7 +347,7 @@ struct ReplaceRegexpImpl
const char * repl_data = reinterpret_cast<const char *>(replacement_data.data() + repl_from);
const size_t repl_length = static_cast<unsigned>(replacement_offsets[i] - repl_from - 1);
re2_st::RE2 searcher(needle, regexp_options);
re2::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());
int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, max_captures);
@ -369,11 +374,11 @@ struct ReplaceRegexpImpl
res_data.reserve(haystack_data.size());
res_offsets.resize(haystack_size);
re2_st::RE2::Options regexp_options;
re2::RE2::Options regexp_options;
/// Don't write error messages to stderr.
regexp_options.set_log_errors(false);
re2_st::RE2 searcher(needle, regexp_options);
re2::RE2 searcher(needle, regexp_options);
if (!searcher.ok())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The pattern argument is not a valid re2 pattern: {}", searcher.error());

View File

@ -50,7 +50,7 @@ bool SlowWithHyperscanChecker::isSlowOneRepeat(std::string_view regexp)
size_t start_pos = 0;
while (start_pos < haystack.size())
{
if (searcher_one_repeat.Match(haystack, start_pos, haystack.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 2))
if (searcher_one_repeat.Match(haystack, start_pos, haystack.size(), re2::RE2::Anchor::UNANCHORED, matches, 2))
{
const auto & match = matches[0];
start_pos = (matches[0].data() - haystack.data()) + match.size(); // new start pos = prefix before match + match length
@ -72,7 +72,7 @@ bool SlowWithHyperscanChecker::isSlowTwoRepeats(std::string_view regexp)
size_t start_pos = 0;
while (start_pos < haystack.size())
{
if (searcher_two_repeats.Match(haystack, start_pos, haystack.size(), re2_st::RE2::Anchor::UNANCHORED, matches, 3))
if (searcher_two_repeats.Match(haystack, start_pos, haystack.size(), re2::RE2::Anchor::UNANCHORED, matches, 3))
{
const auto & match = matches[0];
start_pos = (matches[0].data() - haystack.data()) + match.size(); // new start pos = prefix before match + match length

View File

@ -3,7 +3,14 @@
#include <string_view>
#include <vector>
#include <re2_st/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
{
@ -22,8 +29,8 @@ public:
private:
bool isSlowOneRepeat(std::string_view regexp);
bool isSlowTwoRepeats(std::string_view regexp);
re2_st::RE2 searcher_one_repeat;
re2_st::RE2 searcher_two_repeats;
re2::RE2 searcher_one_repeat;
re2::RE2 searcher_two_repeats;
};
}

View File

@ -55,7 +55,7 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
const ColumnConst * column_pattern = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
const Regexps::Regexp re = Regexps::createRegexp</*is_like*/ false, /*no_capture*/ true, CountMatchesBase::case_insensitive>(column_pattern->getValue<String>());
const OptimizedRegularExpression re = Regexps::createRegexp</*is_like*/ false, /*no_capture*/ true, CountMatchesBase::case_insensitive>(column_pattern->getValue<String>());
OptimizedRegularExpression::MatchVec matches;
const IColumn * column_haystack = arguments[0].column.get();
@ -95,7 +95,7 @@ public:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Error in FunctionCountMatches::getReturnTypeImpl()");
}
static uint64_t countMatches(std::string_view src, const Regexps::Regexp & re, OptimizedRegularExpression::MatchVec & matches)
static uint64_t countMatches(std::string_view src, const OptimizedRegularExpression & re, OptimizedRegularExpression::MatchVec & matches)
{
/// Only one match is required, no need to copy more.
static const unsigned matches_limit = 1;

View File

@ -21,7 +21,7 @@ struct ExtractImpl
res_data.reserve(data.size() / 5);
res_offsets.resize(offsets.size());
const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(pattern);
const OptimizedRegularExpression regexp = Regexps::createRegexp<false, false, false>(pattern);
unsigned capture = regexp.getNumberOfSubpatterns() > 0 ? 1 : 0;
OptimizedRegularExpression::MatchVec matches;

View File

@ -92,7 +92,7 @@ public:
if (needle.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Length of 'needle' argument must be greater than 0.");
const Regexps::Regexp holder = Regexps::createRegexp<false, false, false>(needle);
const OptimizedRegularExpression holder = Regexps::createRegexp<false, false, false>(needle);
const auto & regexp = holder.getRE2();
if (!regexp)

View File

@ -63,7 +63,7 @@ public:
if (needle.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} length of 'needle' argument must be greater than 0.", getName());
const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(needle);
const OptimizedRegularExpression regexp = Regexps::createRegexp<false, false, false>(needle);
const auto & re2 = regexp.getRE2();
if (!re2)
@ -90,7 +90,7 @@ public:
std::string_view current_row = column_haystack->getDataAt(i).toView();
if (re2->Match({current_row.data(), current_row.size()},
0, current_row.size(), re2_st::RE2::UNANCHORED, matched_groups.data(),
0, current_row.size(), re2::RE2::UNANCHORED, matched_groups.data(),
static_cast<int>(matched_groups.size())))
{
// 1 is to exclude group #0 which is whole re match.

View File

@ -132,7 +132,7 @@ private:
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets)
{
const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(pattern);
const OptimizedRegularExpression regexp = Regexps::createRegexp<false, false, false>(pattern);
unsigned capture = regexp.getNumberOfSubpatterns();
if (index < 0 || index >= capture + 1)
throw Exception(
@ -172,7 +172,7 @@ private:
res_data.reserve(data.size() / 5);
res_offsets.reserve(offsets.size());
const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(pattern);
const OptimizedRegularExpression regexp = Regexps::createRegexp<false, false, false>(pattern);
unsigned capture = regexp.getNumberOfSubpatterns();
OptimizedRegularExpression::MatchVec matches;
@ -217,7 +217,7 @@ private:
ColumnString::Chars padded_str;
padded_str.insert(str.begin(), str.end());
const Regexps::Regexp regexp = Regexps::createRegexp<false, false, false>(pattern);
const OptimizedRegularExpression regexp = Regexps::createRegexp<false, false, false>(pattern);
unsigned capture = regexp.getNumberOfSubpatterns();
OptimizedRegularExpression::MatchVec matches;
matches.reserve(capture + 1);

View File

@ -26,8 +26,15 @@
#include "Poco/StreamCopier.h"
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
#include <re2/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
#include <boost/algorithm/string.hpp>
static const int SUCCESS_RESPONSE_MIN = 200;

View File

@ -5,7 +5,15 @@
#include <Common/quoteString.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

@ -2,7 +2,15 @@
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/OvercommitTracker.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

@ -2,11 +2,19 @@
#include <IO/Operators.h>
#include <IO/WriteHelpers.h>
#include <re2/re2.h>
#include <Common/SipHash.h>
#include <Common/quoteString.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

@ -6,9 +6,16 @@
#include <Common/SipHash.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <re2/re2.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

@ -32,9 +32,17 @@
#include <Node.hh>
#include <Schema.hh>
#include <re2/re2.h>
#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

@ -51,8 +51,8 @@ bool RegexpFieldExtractor::parseRow(PeekableReadBuffer & buf)
if (line_size > 0 && buf.position()[line_size - 1] == '\r')
--line_to_match;
bool match = re2_st::RE2::FullMatchN(
re2_st::StringPiece(buf.position(), line_to_match),
bool match = re2::RE2::FullMatchN(
re2::StringPiece(buf.position(), line_to_match),
regexp,
re2_arguments_ptrs.data(),
static_cast<int>(re2_arguments_ptrs.size()));

View File

@ -1,6 +1,5 @@
#pragma once
#include <re2_st/re2.h>
#include <string>
#include <vector>
#include <Core/Block.h>
@ -12,6 +11,14 @@
#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
{
@ -32,12 +39,12 @@ public:
size_t getNumberOfGroups() const { return regexp.NumberOfCapturingGroups(); }
private:
const re2_st::RE2 regexp;
const re2::RE2 regexp;
// The vector of fields extracted from line using regexp.
std::vector<std::string_view> matched_fields;
// These two vectors are needed to use RE2::FullMatchN (function for extracting fields).
std::vector<re2_st::RE2::Arg> re2_arguments;
std::vector<re2_st::RE2::Arg *> re2_arguments_ptrs;
std::vector<re2::RE2::Arg> re2_arguments;
std::vector<re2::RE2::Arg *> re2_arguments_ptrs;
bool skip_unmatched;
};

View File

@ -44,11 +44,18 @@
#include <Poco/String.h>
#include <Poco/Net/SocketAddress.h>
#include <re2/re2.h>
#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

@ -7,7 +7,14 @@
#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
namespace CurrentMetrics
{

View File

@ -5,12 +5,20 @@
#include <Common/StringUtils/StringUtils.h>
#include <base/find_symbols.h>
#include <re2/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

@ -7,7 +7,14 @@
#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 <incbin.h>

View File

@ -9,8 +9,6 @@
#include <Poco/Util/AbstractConfiguration.h>
#include <IO/WriteBufferFromString.h>
#include <re2/re2.h>
namespace DB
{

View File

@ -1,9 +1,17 @@
#include <Storages/HDFS/HDFSCommon.h>
#include <Poco/URI.h>
#include <boost/algorithm/string/replace.hpp>
#include <re2/re2.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
#if USE_HDFS
#include <Common/ShellCommand.h>
#include <Common/Exception.h>

View File

@ -42,12 +42,18 @@
#include <QueryPipeline/Pipe.h>
#include <Poco/URI.h>
#include <re2/re2.h>
#include <re2/stringpiece.h>
#include <hdfs/hdfs.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;

View File

@ -15,7 +15,6 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <re2/re2.h>
#include <azure/storage/common/storage_credential.hpp>
#include <Processors/Transforms/AddingDefaultsTransform.h>
@ -39,6 +38,14 @@
#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;

View File

@ -4,7 +4,6 @@
#if USE_AZURE_BLOB_STORAGE
#include <re2/re2.h>
#include <Storages/IStorage.h>
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
#include <Storages/Cache/SchemaCache.h>
@ -15,6 +14,15 @@
#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

@ -56,12 +56,20 @@
#include <sys/stat.h>
#include <fcntl.h>
#include <unistd.h>
#include <re2/re2.h>
#include <filesystem>
#include <shared_mutex>
#include <cmath>
#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

@ -55,7 +55,6 @@
#include <Common/parseGlobs.h>
#include <Common/quoteString.h>
#include <Common/CurrentMetrics.h>
#include <re2/re2.h>
#include <Processors/ISource.h>
#include <Processors/Sinks/SinkToStorage.h>
@ -64,6 +63,15 @@
#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 fs = std::filesystem;

View File

@ -3,7 +3,6 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Storages/System/StorageSystemCertificates.h>
#include <re2/re2.h>
#include <boost/algorithm/string.hpp>
#include <filesystem>
#include <base/scope_guard.h>
@ -14,6 +13,15 @@
#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

@ -13,7 +13,6 @@ BUILD_DIR=${BUILD_DIR:=./build}
inc="-I. \
-I./contrib/libdivide \
-I./contrib/re2 \
-I${BUILD_DIR}/contrib/re2_st \
-I./contrib/libfarmhash \
-I./contrib/libmetrohash/src \
-I./contrib/double-conversion \