mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Add ability to use custom TLD list
v2: Add a note that top_level_domains_lists aren not applied w/o restart v3: Remove ExtractFirstSignificantSubdomain{Default,Custom}Lookup.h headers v4: TLDListsHolder: remove FIXME for dense_hash_map (this is not significant)
This commit is contained in:
parent
89a3a86b71
commit
916cbd6610
@ -131,6 +131,38 @@ For example:
|
||||
- `cutToFirstSignificantSubdomain('www.tr') = 'www.tr'`.
|
||||
- `cutToFirstSignificantSubdomain('tr') = ''`.
|
||||
|
||||
### cutToFirstSignificantSubdomainCustom {#cuttofirstsignificantsubdomaincustom}
|
||||
|
||||
Same as `cutToFirstSignificantSubdomain` but accept custom TLD list name, useful if:
|
||||
|
||||
- you need fresh TLD list,
|
||||
- or you have custom.
|
||||
|
||||
Configuration example:
|
||||
|
||||
```xml
|
||||
<top_level_domains_lists>
|
||||
<!-- https://publicsuffix.org/list/public_suffix_list.dat -->
|
||||
<public_suffix_list>/path/to/public_suffix_list.dat</public_suffix_list>
|
||||
</top_level_domains_lists>
|
||||
```
|
||||
|
||||
Example:
|
||||
|
||||
- `cutToFirstSignificantSubdomain('https://news.yandex.com.tr/', 'public_suffix_list') = 'yandex.com.tr'`.
|
||||
|
||||
### cutToFirstSignificantSubdomainCustomWithWWW {#cuttofirstsignificantsubdomaincustomwithwww}
|
||||
|
||||
Same as `cutToFirstSignificantSubdomainWithWWW` but accept custom TLD list name.
|
||||
|
||||
### firstSignificantSubdomainCustom {#firstsignificantsubdomaincustom}
|
||||
|
||||
Same as `firstSignificantSubdomain` but accept custom TLD list name.
|
||||
|
||||
### cutToFirstSignificantSubdomainCustomWithWWW {#cuttofirstsignificantsubdomaincustomwithwww}
|
||||
|
||||
Same as `cutToFirstSignificantSubdomainWithWWW` but accept custom TLD list name.
|
||||
|
||||
### port(URL\[, default_port = 0\]) {#port}
|
||||
|
||||
Returns the port or `default_port` if there is no port in the URL (or in case of validation error).
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/getMappedArea.h>
|
||||
#include <Common/remapExecutable.h>
|
||||
#include <Common/TLDListsHolder.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <IO/UseSSL.h>
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
@ -483,6 +484,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
Poco::File(dictionaries_lib_path).createDirectories();
|
||||
}
|
||||
|
||||
/// top_level_domains_lists
|
||||
TLDListsHolder::getInstance().parseConfig(config());
|
||||
|
||||
{
|
||||
Poco::File(path + "data/").createDirectories();
|
||||
Poco::File(path + "metadata/").createDirectories();
|
||||
|
@ -724,6 +724,17 @@
|
||||
<!-- <path_to_regions_names_files>/opt/geo/</path_to_regions_names_files> -->
|
||||
|
||||
|
||||
<!-- Custom TLD lists.
|
||||
Format: <name>/path/to/file</name>
|
||||
|
||||
Changes will not be applied w/o server restart.
|
||||
-->
|
||||
<top_level_domains_lists>
|
||||
<!--
|
||||
<public_suffix_list>/path/to/public_suffix_list.dat</public_suffix_list>
|
||||
-->
|
||||
</top_level_domains_lists>
|
||||
|
||||
<!-- Configuration of external dictionaries. See:
|
||||
https://clickhouse.tech/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts
|
||||
-->
|
||||
|
@ -528,6 +528,7 @@
|
||||
M(559, INVALID_GRPC_QUERY_INFO) \
|
||||
M(560, ZSTD_ENCODER_FAILED) \
|
||||
M(561, ZSTD_DECODER_FAILED) \
|
||||
M(562, TLD_LIST_NOT_FOUND) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
94
src/Common/TLDListsHolder.cpp
Normal file
94
src/Common/TLDListsHolder.cpp
Normal file
@ -0,0 +1,94 @@
|
||||
#include <Common/TLDListsHolder.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <string_view>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TLD_LIST_NOT_FOUND;
|
||||
}
|
||||
|
||||
///
|
||||
/// TLDList
|
||||
///
|
||||
bool TLDList::insert(const StringRef & host)
|
||||
{
|
||||
StringRefHash hash;
|
||||
return tld_container.insert(hash(host)).second;
|
||||
}
|
||||
bool TLDList::has(const StringRef & host) const
|
||||
{
|
||||
StringRefHash hash;
|
||||
return tld_container.has(hash(host));
|
||||
}
|
||||
|
||||
///
|
||||
/// TLDListsHolder
|
||||
///
|
||||
TLDListsHolder & TLDListsHolder::getInstance()
|
||||
{
|
||||
static TLDListsHolder instance;
|
||||
return instance;
|
||||
}
|
||||
TLDListsHolder::TLDListsHolder() = default;
|
||||
|
||||
void TLDListsHolder::parseConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys("top_level_domains_lists", config_keys);
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("TLDListsHolder");
|
||||
|
||||
for (const auto & key : config_keys)
|
||||
{
|
||||
auto path = config.getString("top_level_domains_lists." + key);
|
||||
LOG_TRACE(log, "{} loading from {}", key, path);
|
||||
size_t hosts = parseAndAddTldList(key, path);
|
||||
LOG_INFO(log, "{} was added ({} hosts)", key, hosts);
|
||||
}
|
||||
}
|
||||
|
||||
size_t TLDListsHolder::parseAndAddTldList(const std::string & name, const std::string & path)
|
||||
{
|
||||
TLDList tld_list;
|
||||
|
||||
ReadBufferFromFile in(path);
|
||||
while (!in.eof())
|
||||
{
|
||||
char * newline = find_first_symbols<'\n'>(in.position(), in.buffer().end());
|
||||
if (newline >= in.buffer().end())
|
||||
break;
|
||||
|
||||
std::string_view line(in.position(), newline - in.position());
|
||||
in.position() = newline + 1;
|
||||
|
||||
/// Skip comments
|
||||
if (line.size() > 2 && line[0] == '/' && line[1] == '/')
|
||||
continue;
|
||||
trim(line);
|
||||
/// Skip empty line
|
||||
if (line.empty())
|
||||
continue;
|
||||
tld_list.insert(StringRef{line.data(), line.size()});
|
||||
}
|
||||
|
||||
size_t tld_list_size = tld_list.size();
|
||||
std::lock_guard<std::mutex> lock(tld_lists_map_mutex);
|
||||
tld_lists_map.emplace(name, std::move(tld_list));
|
||||
return tld_list_size;
|
||||
}
|
||||
|
||||
const TLDList & TLDListsHolder::getTldList(const std::string & name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(tld_lists_map_mutex);
|
||||
auto it = tld_lists_map.find(name);
|
||||
if (it == tld_lists_map.end())
|
||||
throw Exception(ErrorCodes::TLD_LIST_NOT_FOUND, "TLD list {} does not exist", name);
|
||||
return it->second;
|
||||
}
|
||||
|
||||
}
|
60
src/Common/TLDListsHolder.h
Normal file
60
src/Common/TLDListsHolder.h
Normal file
@ -0,0 +1,60 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/defines.h>
|
||||
#include <common/StringRef.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <mutex>
|
||||
#include <string>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Custom TLD List
|
||||
/// Unlike tldLookup (which uses gperf) this one uses plain HashSet.
|
||||
class TLDList
|
||||
{
|
||||
public:
|
||||
/// Uses StringRefHash
|
||||
using Container = HashSet<UInt32>;
|
||||
|
||||
/// Return true if the tld_container does not contains such element.
|
||||
bool insert(const StringRef & host);
|
||||
bool has(const StringRef & host) const;
|
||||
size_t size() const { return tld_container.size(); }
|
||||
|
||||
private:
|
||||
Container tld_container;
|
||||
};
|
||||
|
||||
class TLDListsHolder
|
||||
{
|
||||
public:
|
||||
using Map = std::unordered_map<std::string, TLDList>;
|
||||
|
||||
static TLDListsHolder & getInstance();
|
||||
|
||||
/// Parse "top_level_domains_lists" section,
|
||||
/// And add each found dictionary.
|
||||
void parseConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
/// Parse file and add it as a Set to the list of TLDs
|
||||
/// - "//" -- comment,
|
||||
/// - empty lines will be ignored.
|
||||
///
|
||||
/// Example: https://publicsuffix.org/list/public_suffix_list.dat
|
||||
///
|
||||
/// Return size of the list.
|
||||
size_t parseAndAddTldList(const std::string & name, const std::string & path);
|
||||
/// Throws TLD_LIST_NOT_FOUND if list does not exist
|
||||
const TLDList & getTldList(const std::string & name);
|
||||
|
||||
protected:
|
||||
TLDListsHolder();
|
||||
|
||||
std::mutex tld_lists_map_mutex;
|
||||
Map tld_lists_map;
|
||||
};
|
||||
|
||||
}
|
@ -68,6 +68,7 @@ SRCS(
|
||||
StringUtils/StringUtils.cpp
|
||||
StudentTTest.cpp
|
||||
SymbolIndex.cpp
|
||||
TLDListsHolder.cpp
|
||||
TaskStatsInfoGetter.cpp
|
||||
TerminalSize.cpp
|
||||
ThreadFuzzer.cpp
|
||||
|
@ -7,12 +7,27 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct FirstSignificantSubdomainDefaultLookup
|
||||
{
|
||||
bool operator()(const char *src, size_t len) const
|
||||
{
|
||||
return tldLookup::isValid(src, len);
|
||||
}
|
||||
};
|
||||
|
||||
template <bool without_www>
|
||||
struct ExtractFirstSignificantSubdomain
|
||||
{
|
||||
static size_t getReserveLengthForElement() { return 10; }
|
||||
|
||||
static void execute(const Pos data, const size_t size, Pos & res_data, size_t & res_size, Pos * out_domain_end = nullptr)
|
||||
{
|
||||
FirstSignificantSubdomainDefaultLookup loookup;
|
||||
return execute(loookup, data, size, res_data, res_size, out_domain_end);
|
||||
}
|
||||
|
||||
template <class Lookup>
|
||||
static void execute(const Lookup & lookup, const Pos data, const size_t size, Pos & res_data, size_t & res_size, Pos * out_domain_end = nullptr)
|
||||
{
|
||||
res_data = data;
|
||||
res_size = 0;
|
||||
@ -65,7 +80,7 @@ struct ExtractFirstSignificantSubdomain
|
||||
end_of_level_domain = end;
|
||||
}
|
||||
|
||||
if (tldLookup::isValid(last_3_periods[1] + 1, end_of_level_domain - last_3_periods[1] - 1) != nullptr)
|
||||
if (lookup(last_3_periods[1] + 1, end_of_level_domain - last_3_periods[1] - 1))
|
||||
{
|
||||
res_data += last_3_periods[2] + 1 - begin;
|
||||
res_size = last_3_periods[1] - last_3_periods[2] - 1;
|
112
src/Functions/URL/FirstSignificantSubdomainCustomImpl.h
Normal file
112
src/Functions/URL/FirstSignificantSubdomainCustomImpl.h
Normal file
@ -0,0 +1,112 @@
|
||||
#pragma once
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/URL/FunctionsURL.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Common/TLDListsHolder.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
struct FirstSignificantSubdomainCustomtLookup
|
||||
{
|
||||
const TLDList & tld_list;
|
||||
FirstSignificantSubdomainCustomtLookup(const std::string & tld_list_name)
|
||||
: tld_list(TLDListsHolder::getInstance().getTldList(tld_list_name))
|
||||
{
|
||||
}
|
||||
|
||||
bool operator()(const char *pos, size_t len) const
|
||||
{
|
||||
return tld_list.has(StringRef{pos, len});
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Extractor, typename Name>
|
||||
class FunctionCutToFirstSignificantSubdomainCustomImpl : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = Name::name;
|
||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionCutToFirstSignificantSubdomainCustomImpl>(); }
|
||||
|
||||
String getName() const override { return name; }
|
||||
size_t getNumberOfArguments() const override { return 2; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (!isString(arguments[0].type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of first argument of function {}. Must be String.",
|
||||
arguments[0].type->getName(), getName());
|
||||
if (!isString(arguments[1].type))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal type {} of second argument (TLD_list_name) of function {}. Must be String/FixedString.",
|
||||
arguments[1].type->getName(), getName());
|
||||
const auto * column = arguments[1].column.get();
|
||||
if (!column || !checkAndGetColumnConstStringOrFixedString(column))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"The second argument of function {} should be a constant string with the name of the custom TLD",
|
||||
getName());
|
||||
|
||||
return arguments[0].type;
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const ColumnConst * column_tld_list_name = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get());
|
||||
FirstSignificantSubdomainCustomtLookup tld_lookup(column_tld_list_name->getValue<String>());
|
||||
|
||||
/// FIXME: convertToFullColumnIfConst() is suboptimal
|
||||
auto column = arguments[0].column->convertToFullColumnIfConst();
|
||||
if (const ColumnString * col = checkAndGetColumn<ColumnString>(*column))
|
||||
{
|
||||
auto col_res = ColumnString::create();
|
||||
vector(tld_lookup, col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets());
|
||||
return col_res;
|
||||
}
|
||||
else
|
||||
throw Exception(
|
||||
"Illegal column " + arguments[0].column->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
static void vector(FirstSignificantSubdomainCustomtLookup & tld_lookup,
|
||||
const ColumnString::Chars & data, const ColumnString::Offsets & offsets,
|
||||
ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
res_offsets.resize(size);
|
||||
res_data.reserve(size * Extractor::getReserveLengthForElement());
|
||||
|
||||
size_t prev_offset = 0;
|
||||
size_t res_offset = 0;
|
||||
|
||||
/// Matched part.
|
||||
Pos start;
|
||||
size_t length;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Extractor::execute(tld_lookup, reinterpret_cast<const char *>(&data[prev_offset]), offsets[i] - prev_offset - 1, start, length);
|
||||
|
||||
res_data.resize(res_data.size() + length + 1);
|
||||
memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], start, length);
|
||||
res_offset += length + 1;
|
||||
res_data[res_offset - 1] = 0;
|
||||
|
||||
res_offsets[i] = res_offset;
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include "firstSignificantSubdomain.h"
|
||||
#include "ExtractFirstSignificantSubdomain.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
|
43
src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp
Normal file
43
src/Functions/URL/cutToFirstSignificantSubdomainCustom.cpp
Normal file
@ -0,0 +1,43 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include "ExtractFirstSignificantSubdomain.h"
|
||||
#include "FirstSignificantSubdomainCustomImpl.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <bool without_www>
|
||||
struct CutToFirstSignificantSubdomainCustom
|
||||
{
|
||||
static size_t getReserveLengthForElement() { return 15; }
|
||||
|
||||
static void execute(FirstSignificantSubdomainCustomtLookup & tld_lookup, const Pos data, const size_t size, Pos & res_data, size_t & res_size)
|
||||
{
|
||||
res_data = data;
|
||||
res_size = 0;
|
||||
|
||||
Pos tmp_data;
|
||||
size_t tmp_length;
|
||||
Pos domain_end;
|
||||
ExtractFirstSignificantSubdomain<without_www>::execute(tld_lookup, data, size, tmp_data, tmp_length, &domain_end);
|
||||
|
||||
if (tmp_length == 0)
|
||||
return;
|
||||
|
||||
res_data = tmp_data;
|
||||
res_size = domain_end - tmp_data;
|
||||
}
|
||||
};
|
||||
|
||||
struct NameCutToFirstSignificantSubdomainCustom { static constexpr auto name = "cutToFirstSignificantSubdomainCustom"; };
|
||||
using FunctionCutToFirstSignificantSubdomainCustom = FunctionCutToFirstSignificantSubdomainCustomImpl<CutToFirstSignificantSubdomainCustom<true>, NameCutToFirstSignificantSubdomainCustom>;
|
||||
|
||||
struct NameCutToFirstSignificantSubdomainCustomWithWWW { static constexpr auto name = "cutToFirstSignificantSubdomainCustomWithWWW"; };
|
||||
using FunctionCutToFirstSignificantSubdomainCustomWithWWW = FunctionCutToFirstSignificantSubdomainCustomImpl<CutToFirstSignificantSubdomainCustom<false>, NameCutToFirstSignificantSubdomainCustomWithWWW>;
|
||||
|
||||
void registerFunctionCutToFirstSignificantSubdomainCustom(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionCutToFirstSignificantSubdomainCustom>();
|
||||
factory.registerFunction<FunctionCutToFirstSignificantSubdomainCustomWithWWW>();
|
||||
}
|
||||
|
||||
}
|
@ -1,12 +1,13 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringToString.h>
|
||||
#include "firstSignificantSubdomain.h"
|
||||
#include "ExtractFirstSignificantSubdomain.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameFirstSignificantSubdomain { static constexpr auto name = "firstSignificantSubdomain"; };
|
||||
|
||||
using FunctionFirstSignificantSubdomain = FunctionStringToString<ExtractSubstringImpl<ExtractFirstSignificantSubdomain<true>>, NameFirstSignificantSubdomain>;
|
||||
|
||||
void registerFunctionFirstSignificantSubdomain(FunctionFactory & factory)
|
||||
|
18
src/Functions/URL/firstSignificantSubdomainCustom.cpp
Normal file
18
src/Functions/URL/firstSignificantSubdomainCustom.cpp
Normal file
@ -0,0 +1,18 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include "ExtractFirstSignificantSubdomain.h"
|
||||
#include "FirstSignificantSubdomainCustomImpl.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct NameFirstSignificantSubdomainCustom { static constexpr auto name = "firstSignificantSubdomainCustom"; };
|
||||
|
||||
using FunctionFirstSignificantSubdomainCustom = FunctionCutToFirstSignificantSubdomainCustomImpl<ExtractFirstSignificantSubdomain<true>, NameFirstSignificantSubdomainCustom>;
|
||||
|
||||
void registerFunctionFirstSignificantSubdomainCustom(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionFirstSignificantSubdomainCustom>();
|
||||
}
|
||||
|
||||
}
|
@ -7,6 +7,7 @@ void registerFunctionProtocol(FunctionFactory & factory);
|
||||
void registerFunctionDomain(FunctionFactory & factory);
|
||||
void registerFunctionDomainWithoutWWW(FunctionFactory & factory);
|
||||
void registerFunctionFirstSignificantSubdomain(FunctionFactory & factory);
|
||||
void registerFunctionFirstSignificantSubdomainCustom(FunctionFactory & factory);
|
||||
void registerFunctionTopLevelDomain(FunctionFactory & factory);
|
||||
void registerFunctionPort(FunctionFactory & factory);
|
||||
void registerFunctionPath(FunctionFactory & factory);
|
||||
@ -20,6 +21,7 @@ void registerFunctionExtractURLParameterNames(FunctionFactory & factory);
|
||||
void registerFunctionURLHierarchy(FunctionFactory & factory);
|
||||
void registerFunctionURLPathHierarchy(FunctionFactory & factory);
|
||||
void registerFunctionCutToFirstSignificantSubdomain(FunctionFactory & factory);
|
||||
void registerFunctionCutToFirstSignificantSubdomainCustom(FunctionFactory & factory);
|
||||
void registerFunctionCutWWW(FunctionFactory & factory);
|
||||
void registerFunctionCutQueryString(FunctionFactory & factory);
|
||||
void registerFunctionCutFragment(FunctionFactory & factory);
|
||||
@ -34,6 +36,7 @@ void registerFunctionsURL(FunctionFactory & factory)
|
||||
registerFunctionDomain(factory);
|
||||
registerFunctionDomainWithoutWWW(factory);
|
||||
registerFunctionFirstSignificantSubdomain(factory);
|
||||
registerFunctionFirstSignificantSubdomainCustom(factory);
|
||||
registerFunctionTopLevelDomain(factory);
|
||||
registerFunctionPort(factory);
|
||||
registerFunctionPath(factory);
|
||||
@ -47,6 +50,7 @@ void registerFunctionsURL(FunctionFactory & factory)
|
||||
registerFunctionURLHierarchy(factory);
|
||||
registerFunctionURLPathHierarchy(factory);
|
||||
registerFunctionCutToFirstSignificantSubdomain(factory);
|
||||
registerFunctionCutToFirstSignificantSubdomainCustom(factory);
|
||||
registerFunctionCutWWW(factory);
|
||||
registerFunctionCutQueryString(factory);
|
||||
registerFunctionCutFragment(factory);
|
||||
|
@ -1,5 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <cstdlib>
|
||||
|
||||
// Definition of the class generated by gperf, present on gperf/tldLookup.gperf
|
||||
class TopLevelDomainLookupHash
|
||||
{
|
||||
|
@ -80,6 +80,7 @@ SRCS(
|
||||
URL/cutQueryString.cpp
|
||||
URL/cutQueryStringAndFragment.cpp
|
||||
URL/cutToFirstSignificantSubdomain.cpp
|
||||
URL/cutToFirstSignificantSubdomainCustom.cpp
|
||||
URL/cutURLParameter.cpp
|
||||
URL/cutWWW.cpp
|
||||
URL/decodeURLComponent.cpp
|
||||
@ -89,6 +90,7 @@ SRCS(
|
||||
URL/extractURLParameterNames.cpp
|
||||
URL/extractURLParameters.cpp
|
||||
URL/firstSignificantSubdomain.cpp
|
||||
URL/firstSignificantSubdomainCustom.cpp
|
||||
URL/fragment.cpp
|
||||
URL/netloc.cpp
|
||||
URL/path.cpp
|
||||
|
Loading…
Reference in New Issue
Block a user