mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge remote-tracking branch 'origin/master' into HEAD
This commit is contained in:
commit
a2886cdfa5
@ -64,7 +64,14 @@ function stop_server
|
||||
function start_server
|
||||
{
|
||||
set -m # Spawn server in its own process groups
|
||||
clickhouse-server --config-file="$FASTTEST_DATA/config.xml" -- --path "$FASTTEST_DATA" --user_files_path "$FASTTEST_DATA/user_files" &>> "$FASTTEST_OUTPUT/server.log" &
|
||||
local opts=(
|
||||
--config-file="$FASTTEST_DATA/config.xml"
|
||||
--
|
||||
--path "$FASTTEST_DATA"
|
||||
--user_files_path "$FASTTEST_DATA/user_files"
|
||||
--top_level_domains_path "$FASTTEST_DATA/top_level_domains"
|
||||
)
|
||||
clickhouse-server "${opts[@]}" &>> "$FASTTEST_OUTPUT/server.log" &
|
||||
server_pid=$!
|
||||
set +m
|
||||
|
||||
|
@ -53,4 +53,3 @@ COPY * /
|
||||
CMD ["bash", "-c", "node=$((RANDOM % $(numactl --hardware | sed -n 's/^.*available:\\(.*\\)nodes.*$/\\1/p'))); echo Will bind to NUMA node $node; numactl --cpunodebind=$node --membind=$node /entrypoint.sh"]
|
||||
|
||||
# docker run --network=host --volume <workspace>:/workspace --volume=<output>:/output -e PR_TO_TEST=<> -e SHA_TO_TEST=<> yandex/clickhouse-performance-comparison
|
||||
|
||||
|
@ -55,6 +55,7 @@ function configure
|
||||
# server *config* directives overrides
|
||||
--path db0
|
||||
--user_files_path db0/user_files
|
||||
--top_level_domains_path /top_level_domains
|
||||
--tcp_port $LEFT_SERVER_PORT
|
||||
)
|
||||
left/clickhouse-server "${setup_left_server_opts[@]}" &> setup-server-log.log &
|
||||
@ -102,6 +103,7 @@ function restart
|
||||
# server *config* directives overrides
|
||||
--path left/db
|
||||
--user_files_path left/db/user_files
|
||||
--top_level_domains_path /top_level_domains
|
||||
--tcp_port $LEFT_SERVER_PORT
|
||||
)
|
||||
left/clickhouse-server "${left_server_opts[@]}" &>> left-server-log.log &
|
||||
@ -116,6 +118,7 @@ function restart
|
||||
# server *config* directives overrides
|
||||
--path right/db
|
||||
--user_files_path right/db/user_files
|
||||
--top_level_domains_path /top_level_domains
|
||||
--tcp_port $RIGHT_SERVER_PORT
|
||||
)
|
||||
right/clickhouse-server "${right_server_opts[@]}" &>> right-server-log.log &
|
||||
|
@ -0,0 +1,5 @@
|
||||
<yandex>
|
||||
<top_level_domains_lists>
|
||||
<public_suffix_list>public_suffix_list.dat</public_suffix_list>
|
||||
</top_level_domains_lists>
|
||||
</yandex>
|
File diff suppressed because it is too large
Load Diff
@ -131,6 +131,40 @@ 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_path>/var/lib/clickhouse/top_level_domains/</top_level_domains_path> -->
|
||||
<top_level_domains_lists>
|
||||
<!-- https://publicsuffix.org/list/public_suffix_list.dat -->
|
||||
<public_suffix_list>public_suffix_list.dat</public_suffix_list>
|
||||
<!-- NOTE: path is under top_level_domains_path -->
|
||||
</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).
|
||||
|
@ -297,7 +297,7 @@ FORMAT Null;
|
||||
**Смотрите также**
|
||||
|
||||
- [Секция JOIN](../../sql-reference/statements/select/join.md#select-join)
|
||||
- [Движоy таблиц Join](../../engines/table-engines/special/join.md)
|
||||
- [Движок таблиц Join](../../engines/table-engines/special/join.md)
|
||||
|
||||
## max_partitions_per_insert_block {#max-partitions-per-insert-block}
|
||||
|
||||
|
@ -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>
|
||||
@ -542,6 +543,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
Poco::File(dictionaries_lib_path).createDirectories();
|
||||
}
|
||||
|
||||
/// top_level_domains_lists
|
||||
{
|
||||
const std::string & top_level_domains_path = config().getString("top_level_domains_path", path + "top_level_domains/") + "/";
|
||||
TLDListsHolder::getInstance().parseConfig(top_level_domains_path, config());
|
||||
}
|
||||
|
||||
{
|
||||
Poco::File(path + "data/").createDirectories();
|
||||
Poco::File(path + "metadata/").createDirectories();
|
||||
|
@ -4,4 +4,5 @@
|
||||
<user_files_path replace="replace">./user_files/</user_files_path>
|
||||
<format_schema_path replace="replace">./format_schemas/</format_schema_path>
|
||||
<access_control_path replace="replace">./access/</access_control_path>
|
||||
<top_level_domains_path replace="replace">./top_level_domains/</top_level_domains_path>
|
||||
</yandex>
|
||||
|
@ -724,6 +724,19 @@
|
||||
<!-- <path_to_regions_names_files>/opt/geo/</path_to_regions_names_files> -->
|
||||
|
||||
|
||||
<!-- <top_level_domains_path>/var/lib/clickhouse/top_level_domains/</top_level_domains_path> -->
|
||||
<!-- Custom TLD lists.
|
||||
Format: <name>/path/to/file</name>
|
||||
|
||||
Changes will not be applied w/o server restart.
|
||||
Path to the list is under top_level_domains_path (see above).
|
||||
-->
|
||||
<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) \
|
||||
|
101
src/Common/HashTable/StringHashSet.h
Normal file
101
src/Common/HashTable/StringHashSet.h
Normal file
@ -0,0 +1,101 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Common/HashTable/HashTableAllocator.h>
|
||||
#include <Common/HashTable/StringHashTable.h>
|
||||
|
||||
template <typename Key>
|
||||
struct StringHashSetCell : public HashTableCell<Key, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashTableCell<Key, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
|
||||
VoidMapped void_map;
|
||||
VoidMapped & getMapped() { return void_map; }
|
||||
const VoidMapped & getMapped() const { return void_map; }
|
||||
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
};
|
||||
|
||||
template <>
|
||||
struct StringHashSetCell<StringKey16> : public HashTableCell<StringKey16, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashTableCell<StringKey16, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
|
||||
VoidMapped void_map;
|
||||
VoidMapped & getMapped() { return void_map; }
|
||||
const VoidMapped & getMapped() const { return void_map; }
|
||||
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
|
||||
bool isZero(const HashTableNoState & state) const { return isZero(this->key, state); }
|
||||
// Zero means unoccupied cells in hash table. Use key with last word = 0 as
|
||||
// zero keys, because such keys are unrepresentable (no way to encode length).
|
||||
static bool isZero(const StringKey16 & key_, const HashTableNoState &)
|
||||
{ return key_.high == 0; }
|
||||
void setZero() { this->key.high = 0; }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct StringHashSetCell<StringKey24> : public HashTableCell<StringKey24, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashTableCell<StringKey24, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
|
||||
VoidMapped void_map;
|
||||
VoidMapped & getMapped() { return void_map; }
|
||||
const VoidMapped & getMapped() const { return void_map; }
|
||||
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
|
||||
bool isZero(const HashTableNoState & state) const { return isZero(this->key, state); }
|
||||
// Zero means unoccupied cells in hash table. Use key with last word = 0 as
|
||||
// zero keys, because such keys are unrepresentable (no way to encode length).
|
||||
static bool isZero(const StringKey24 & key_, const HashTableNoState &)
|
||||
{ return key_.c == 0; }
|
||||
void setZero() { this->key.c = 0; }
|
||||
};
|
||||
|
||||
template <>
|
||||
struct StringHashSetCell<StringRef> : public HashSetCellWithSavedHash<StringRef, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashSetCellWithSavedHash<StringRef, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
|
||||
VoidMapped void_map;
|
||||
VoidMapped & getMapped() { return void_map; }
|
||||
const VoidMapped & getMapped() const { return void_map; }
|
||||
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
};
|
||||
|
||||
template <typename Allocator>
|
||||
struct StringHashSetSubMaps
|
||||
{
|
||||
using T0 = StringHashTableEmpty<StringHashSetCell<StringRef>>;
|
||||
using T1 = HashSetTable<StringKey8, StringHashSetCell<StringKey8>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
using T2 = HashSetTable<StringKey16, StringHashSetCell<StringKey16>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
using T3 = HashSetTable<StringKey24, StringHashSetCell<StringKey24>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
using Ts = HashSetTable<StringRef, StringHashSetCell<StringRef>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
};
|
||||
|
||||
template <typename Allocator = HashTableAllocator>
|
||||
class StringHashSet : public StringHashTable<StringHashSetSubMaps<Allocator>>
|
||||
{
|
||||
public:
|
||||
using Key = StringRef;
|
||||
using Base = StringHashTable<StringHashSetSubMaps<Allocator>>;
|
||||
using Self = StringHashSet;
|
||||
using LookupResult = typename Base::LookupResult;
|
||||
|
||||
using Base::Base;
|
||||
|
||||
template <typename KeyHolder>
|
||||
void ALWAYS_INLINE emplace(KeyHolder && key_holder, bool & inserted)
|
||||
{
|
||||
LookupResult it;
|
||||
Base::emplace(key_holder, it, inserted);
|
||||
}
|
||||
|
||||
};
|
@ -212,7 +212,7 @@ public:
|
||||
using LookupResult = StringHashTableLookupResult<typename cell_type::mapped_type>;
|
||||
using ConstLookupResult = StringHashTableLookupResult<const typename cell_type::mapped_type>;
|
||||
|
||||
StringHashTable() {}
|
||||
StringHashTable() = default;
|
||||
|
||||
StringHashTable(size_t reserve_for_num_elements)
|
||||
: m1{reserve_for_num_elements / 4}
|
||||
@ -222,8 +222,15 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
StringHashTable(StringHashTable && rhs) { *this = std::move(rhs); }
|
||||
~StringHashTable() {}
|
||||
StringHashTable(StringHashTable && rhs)
|
||||
: m1(std::move(rhs.m1))
|
||||
, m2(std::move(rhs.m2))
|
||||
, m3(std::move(rhs.m3))
|
||||
, ms(std::move(rhs.ms))
|
||||
{
|
||||
}
|
||||
|
||||
~StringHashTable() = default;
|
||||
|
||||
public:
|
||||
// Dispatch is written in a way that maximizes the performance:
|
||||
|
106
src/Common/TLDListsHolder.cpp
Normal file
106
src/Common/TLDListsHolder.cpp
Normal file
@ -0,0 +1,106 @@
|
||||
#include <Common/TLDListsHolder.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <string_view>
|
||||
#include <unordered_set>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TLD_LIST_NOT_FOUND;
|
||||
}
|
||||
|
||||
///
|
||||
/// TLDList
|
||||
///
|
||||
TLDList::TLDList(size_t size)
|
||||
: tld_container(size)
|
||||
, pool(std::make_unique<Arena>(10 << 20))
|
||||
{}
|
||||
bool TLDList::insert(const StringRef & host)
|
||||
{
|
||||
bool inserted;
|
||||
tld_container.emplace(DB::ArenaKeyHolder{host, *pool}, inserted);
|
||||
return inserted;
|
||||
}
|
||||
bool TLDList::has(const StringRef & host) const
|
||||
{
|
||||
return tld_container.has(host);
|
||||
}
|
||||
|
||||
///
|
||||
/// TLDListsHolder
|
||||
///
|
||||
TLDListsHolder & TLDListsHolder::getInstance()
|
||||
{
|
||||
static TLDListsHolder instance;
|
||||
return instance;
|
||||
}
|
||||
TLDListsHolder::TLDListsHolder() = default;
|
||||
|
||||
void TLDListsHolder::parseConfig(const std::string & top_level_domains_path, 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)
|
||||
{
|
||||
const std::string & path = top_level_domains_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)
|
||||
{
|
||||
std::unordered_set<std::string> tld_list_tmp;
|
||||
|
||||
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_tmp.emplace(line);
|
||||
}
|
||||
|
||||
TLDList tld_list(tld_list_tmp.size());
|
||||
for (const auto & host : tld_list_tmp)
|
||||
{
|
||||
StringRef host_ref{host.data(), host.size()};
|
||||
tld_list.insert(host_ref);
|
||||
}
|
||||
|
||||
size_t tld_list_size = tld_list.size();
|
||||
std::lock_guard<std::mutex> lock(tld_lists_map_mutex);
|
||||
tld_lists_map.insert(std::make_pair(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;
|
||||
}
|
||||
|
||||
}
|
65
src/Common/TLDListsHolder.h
Normal file
65
src/Common/TLDListsHolder.h
Normal file
@ -0,0 +1,65 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/defines.h>
|
||||
#include <common/StringRef.h>
|
||||
#include <Common/HashTable/StringHashSet.h>
|
||||
#include <Common/Arena.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 StringHashSet.
|
||||
class TLDList
|
||||
{
|
||||
public:
|
||||
using Container = StringHashSet<>;
|
||||
|
||||
TLDList(size_t size);
|
||||
|
||||
/// Return true if the tld_container does not contains such element.
|
||||
bool insert(const StringRef & host);
|
||||
/// Check is there such TLD
|
||||
bool has(const StringRef & host) const;
|
||||
size_t size() const { return tld_container.size(); }
|
||||
|
||||
private:
|
||||
Container tld_container;
|
||||
std::unique_ptr<Arena> pool;
|
||||
};
|
||||
|
||||
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 std::string & top_level_domains_path, 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;
|
||||
};
|
||||
|
||||
}
|
@ -427,7 +427,7 @@ struct TestKeeperStorageMultiRequest final : public TestKeeperStorageRequest
|
||||
|
||||
for (const auto & sub_request : request.requests)
|
||||
{
|
||||
auto sub_zk_request = dynamic_pointer_cast<Coordination::ZooKeeperRequest>(sub_request);
|
||||
auto sub_zk_request = std::dynamic_pointer_cast<Coordination::ZooKeeperRequest>(sub_request);
|
||||
if (sub_zk_request->getOpNum() == Coordination::OpNum::Create)
|
||||
{
|
||||
concrete_requests.push_back(std::make_shared<TestKeeperStorageCreateRequest>(sub_zk_request));
|
||||
|
@ -68,6 +68,7 @@ SRCS(
|
||||
StringUtils/StringUtils.cpp
|
||||
StudentTTest.cpp
|
||||
SymbolIndex.cpp
|
||||
TLDListsHolder.cpp
|
||||
TaskStatsInfoGetter.cpp
|
||||
TerminalSize.cpp
|
||||
ThreadFuzzer.cpp
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include "Disks/DiskFactory.h"
|
||||
|
||||
#include <random>
|
||||
#include <optional>
|
||||
#include <utility>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
@ -326,11 +327,19 @@ namespace
|
||||
const String & bucket_,
|
||||
Metadata metadata_,
|
||||
const String & s3_path_,
|
||||
std::optional<DiskS3::ObjectMetadata> object_metadata_,
|
||||
bool is_multipart,
|
||||
size_t min_upload_part_size,
|
||||
size_t buf_size_)
|
||||
: WriteBufferFromFileBase(buf_size_, nullptr, 0)
|
||||
, impl(WriteBufferFromS3(client_ptr_, bucket_, metadata_.s3_root_path + s3_path_, min_upload_part_size, is_multipart, buf_size_))
|
||||
, impl(WriteBufferFromS3(
|
||||
client_ptr_,
|
||||
bucket_,
|
||||
metadata_.s3_root_path + s3_path_,
|
||||
min_upload_part_size,
|
||||
is_multipart,
|
||||
std::move(object_metadata_),
|
||||
buf_size_))
|
||||
, metadata(std::move(metadata_))
|
||||
, s3_path(s3_path_)
|
||||
{
|
||||
@ -522,7 +531,8 @@ DiskS3::DiskS3(
|
||||
String metadata_path_,
|
||||
size_t min_upload_part_size_,
|
||||
size_t min_multi_part_upload_size_,
|
||||
size_t min_bytes_for_seek_)
|
||||
size_t min_bytes_for_seek_,
|
||||
bool send_metadata_)
|
||||
: IDisk(std::make_unique<AsyncExecutor>())
|
||||
, name(std::move(name_))
|
||||
, client(std::move(client_))
|
||||
@ -533,6 +543,7 @@ DiskS3::DiskS3(
|
||||
, min_upload_part_size(min_upload_part_size_)
|
||||
, min_multi_part_upload_size(min_multi_part_upload_size_)
|
||||
, min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, send_metadata(send_metadata_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -653,6 +664,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
}
|
||||
/// Path to store new S3 object.
|
||||
auto s3_path = getRandomName();
|
||||
auto object_metadata = createObjectMetadata(path);
|
||||
bool is_multipart = estimated_size >= min_multi_part_upload_size;
|
||||
if (!exist || mode == WriteMode::Rewrite)
|
||||
{
|
||||
@ -664,9 +676,9 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
/// Save empty metadata to disk to have ability to get file size while buffer is not finalized.
|
||||
metadata.save();
|
||||
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_root_path + s3_path);
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Write to file by path: {}. New S3 path: {}", backQuote(metadata_path + path), s3_root_path + s3_path);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, is_multipart, min_upload_part_size, buf_size);
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, object_metadata, is_multipart, min_upload_part_size, buf_size);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -675,7 +687,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskS3::writeFile(const String & path,
|
||||
LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.",
|
||||
backQuote(metadata_path + path), s3_root_path + s3_path, metadata.s3_objects.size());
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, is_multipart, min_upload_part_size, buf_size);
|
||||
return std::make_unique<WriteIndirectBufferFromS3>(client, bucket, metadata, s3_path, object_metadata, is_multipart, min_upload_part_size, buf_size);
|
||||
}
|
||||
}
|
||||
|
||||
@ -847,4 +859,12 @@ void DiskS3::shutdown()
|
||||
client->DisableRequestProcessing();
|
||||
}
|
||||
|
||||
std::optional<DiskS3::ObjectMetadata> DiskS3::createObjectMetadata(const String & path) const
|
||||
{
|
||||
if (send_metadata)
|
||||
return (DiskS3::ObjectMetadata){{"path", path}};
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,6 +19,8 @@ namespace DB
|
||||
class DiskS3 : public IDisk
|
||||
{
|
||||
public:
|
||||
using ObjectMetadata = std::map<std::string, std::string>;
|
||||
|
||||
friend class DiskS3Reservation;
|
||||
|
||||
class AwsS3KeyKeeper;
|
||||
@ -32,7 +34,8 @@ public:
|
||||
String metadata_path_,
|
||||
size_t min_upload_part_size_,
|
||||
size_t min_multi_part_upload_size_,
|
||||
size_t min_bytes_for_seek_);
|
||||
size_t min_bytes_for_seek_,
|
||||
bool send_metadata_);
|
||||
|
||||
const String & getName() const override { return name; }
|
||||
|
||||
@ -116,6 +119,7 @@ private:
|
||||
void removeMeta(const String & path, AwsS3KeyKeeper & keys);
|
||||
void removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys);
|
||||
void removeAws(const AwsS3KeyKeeper & keys);
|
||||
std::optional<ObjectMetadata> createObjectMetadata(const String & path) const;
|
||||
|
||||
private:
|
||||
const String name;
|
||||
@ -127,6 +131,7 @@ private:
|
||||
size_t min_upload_part_size;
|
||||
size_t min_multi_part_upload_size;
|
||||
size_t min_bytes_for_seek;
|
||||
bool send_metadata;
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
|
@ -149,7 +149,8 @@ void registerDiskS3(DiskFactory & factory)
|
||||
metadata_path,
|
||||
context.getSettingsRef().s3_min_upload_part_size,
|
||||
config.getUInt64(config_prefix + ".min_multi_part_upload_size", 10 * 1024 * 1024),
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024));
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getBool(config_prefix + ".send_object_metadata", false));
|
||||
|
||||
/// This code is used only to check access to the corresponding disk.
|
||||
if (!config.getBool(config_prefix + ".skip_access_check", false))
|
||||
|
@ -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
|
||||
|
@ -43,11 +43,13 @@ WriteBufferFromS3::WriteBufferFromS3(
|
||||
const String & key_,
|
||||
size_t minimum_upload_part_size_,
|
||||
bool is_multipart_,
|
||||
std::optional<std::map<String, String>> object_metadata_,
|
||||
size_t buffer_size_)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buffer_size_, nullptr, 0)
|
||||
, is_multipart(is_multipart_)
|
||||
, bucket(bucket_)
|
||||
, key(key_)
|
||||
, object_metadata(std::move(object_metadata_))
|
||||
, client_ptr(std::move(client_ptr_))
|
||||
, minimum_upload_part_size{minimum_upload_part_size_}
|
||||
, temporary_buffer{std::make_unique<WriteBufferFromOwnString>()}
|
||||
@ -116,6 +118,8 @@ void WriteBufferFromS3::initiate()
|
||||
Aws::S3::Model::CreateMultipartUploadRequest req;
|
||||
req.SetBucket(bucket);
|
||||
req.SetKey(key);
|
||||
if (object_metadata.has_value())
|
||||
req.SetMetadata(object_metadata.value());
|
||||
|
||||
auto outcome = client_ptr->CreateMultipartUpload(req);
|
||||
|
||||
@ -217,6 +221,8 @@ void WriteBufferFromS3::complete()
|
||||
Aws::S3::Model::PutObjectRequest req;
|
||||
req.SetBucket(bucket);
|
||||
req.SetKey(key);
|
||||
if (object_metadata.has_value())
|
||||
req.SetMetadata(object_metadata.value());
|
||||
|
||||
/// This could be improved using an adapter to WriteBuffer.
|
||||
const std::shared_ptr<Aws::IOStream> input_data = Aws::MakeShared<Aws::StringStream>("temporary buffer", temporary_buffer->str());
|
||||
|
@ -28,6 +28,7 @@ private:
|
||||
|
||||
String bucket;
|
||||
String key;
|
||||
std::optional<std::map<String, String>> object_metadata;
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr;
|
||||
size_t minimum_upload_part_size;
|
||||
std::unique_ptr<WriteBufferFromOwnString> temporary_buffer;
|
||||
@ -47,6 +48,7 @@ public:
|
||||
const String & key_,
|
||||
size_t minimum_upload_part_size_,
|
||||
bool is_multipart,
|
||||
std::optional<std::map<String, String>> object_metadata_ = std::nullopt,
|
||||
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
|
||||
void nextImpl() override;
|
||||
|
@ -180,7 +180,7 @@ static inline std::tuple<NamesAndTypesList, NamesAndTypesList, NamesAndTypesList
|
||||
if (const auto & function = index_expression->as<ASTFunction>())
|
||||
{
|
||||
/// column_name(int64 literal)
|
||||
if (columns_name_set.contains(function->name) && function->arguments->children.size() == 1)
|
||||
if (columns_name_set.count(function->name) && function->arguments->children.size() == 1)
|
||||
{
|
||||
const auto & prefix_limit = function->arguments->children[0]->as<ASTLiteral>();
|
||||
|
||||
|
@ -125,12 +125,60 @@ struct CustomizeAggregateFunctionsSuffixData
|
||||
{
|
||||
auto properties = instance.tryGetProperties(func.name);
|
||||
if (properties && !properties->returns_default_when_only_null)
|
||||
func.name = func.name + customized_func_suffix;
|
||||
{
|
||||
func.name += customized_func_suffix;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Used to rewrite aggregate functions with -OrNull suffix in some cases, such as sumIfOrNull, we shoule rewrite to sumOrNullIf
|
||||
struct CustomizeAggregateFunctionsMoveSuffixData
|
||||
{
|
||||
using TypeToVisit = ASTFunction;
|
||||
|
||||
const String & customized_func_suffix;
|
||||
|
||||
String moveSuffixAhead(const String & name) const
|
||||
{
|
||||
auto prefix = name.substr(0, name.size() - customized_func_suffix.size());
|
||||
|
||||
auto prefix_size = prefix.size();
|
||||
|
||||
if (endsWith(prefix, "MergeState"))
|
||||
return prefix.substr(0, prefix_size - 10) + customized_func_suffix + "MergeState";
|
||||
|
||||
if (endsWith(prefix, "Merge"))
|
||||
return prefix.substr(0, prefix_size - 5) + customized_func_suffix + "Merge";
|
||||
|
||||
if (endsWith(prefix, "State"))
|
||||
return prefix.substr(0, prefix_size - 5) + customized_func_suffix + "State";
|
||||
|
||||
if (endsWith(prefix, "If"))
|
||||
return prefix.substr(0, prefix_size - 2) + customized_func_suffix + "If";
|
||||
|
||||
return name;
|
||||
}
|
||||
|
||||
void visit(ASTFunction & func, ASTPtr &) const
|
||||
{
|
||||
const auto & instance = AggregateFunctionFactory::instance();
|
||||
if (instance.isAggregateFunctionName(func.name))
|
||||
{
|
||||
if (endsWith(func.name, customized_func_suffix))
|
||||
{
|
||||
auto properties = instance.tryGetProperties(func.name);
|
||||
if (properties && !properties->returns_default_when_only_null)
|
||||
{
|
||||
func.name = moveSuffixAhead(func.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
using CustomizeAggregateFunctionsOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsSuffixData>, true>;
|
||||
using CustomizeAggregateFunctionsMoveOrNullVisitor = InDepthNodeVisitor<OneTypeMatcher<CustomizeAggregateFunctionsMoveSuffixData>, true>;
|
||||
|
||||
/// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form.
|
||||
/// Expand asterisks and qualified asterisks with column names.
|
||||
@ -753,6 +801,10 @@ void TreeRewriter::normalize(ASTPtr & query, Aliases & aliases, const Settings &
|
||||
CustomizeAggregateFunctionsOrNullVisitor(data_or_null).visit(query);
|
||||
}
|
||||
|
||||
/// Move -OrNull suffix ahead, this should execute after add -OrNull suffix
|
||||
CustomizeAggregateFunctionsMoveOrNullVisitor::Data data_or_null{"OrNull"};
|
||||
CustomizeAggregateFunctionsMoveOrNullVisitor(data_or_null).visit(query);
|
||||
|
||||
/// Creates a dictionary `aliases`: alias -> ASTPtr
|
||||
QueryAliasesVisitor(aliases).visit(query);
|
||||
|
||||
|
@ -29,6 +29,9 @@ target_link_libraries (string_hash_map PRIVATE dbms)
|
||||
add_executable (string_hash_map_aggregation string_hash_map.cpp)
|
||||
target_link_libraries (string_hash_map_aggregation PRIVATE dbms)
|
||||
|
||||
add_executable (string_hash_set string_hash_set.cpp)
|
||||
target_link_libraries (string_hash_set PRIVATE dbms)
|
||||
|
||||
add_executable (two_level_hash_map two_level_hash_map.cpp)
|
||||
target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARSEHASH_INCLUDE_DIR})
|
||||
target_link_libraries (two_level_hash_map PRIVATE dbms)
|
||||
|
83
src/Interpreters/tests/string_hash_set.cpp
Normal file
83
src/Interpreters/tests/string_hash_set.cpp
Normal file
@ -0,0 +1,83 @@
|
||||
#include <iomanip>
|
||||
#include <iostream>
|
||||
#include <vector>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <common/types.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Interpreters/AggregationCommon.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Common/HashTable/HashTableKeyHolder.h>
|
||||
#include <Common/HashTable/StringHashSet.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <common/StringRef.h>
|
||||
|
||||
/// NOTE: see string_hash_map.cpp for usage example
|
||||
|
||||
template <typename Set>
|
||||
void NO_INLINE bench(const std::vector<StringRef> & data, DB::Arena & pool, const char * name)
|
||||
{
|
||||
std::cerr << "method " << name << std::endl;
|
||||
for (auto t = 0ul; t < 7; ++t)
|
||||
{
|
||||
Stopwatch watch;
|
||||
Set set;
|
||||
typename Set::LookupResult it;
|
||||
bool inserted;
|
||||
|
||||
for (const auto & value : data)
|
||||
{
|
||||
if constexpr (std::is_same_v<StringHashSet<>, Set>)
|
||||
set.emplace(DB::ArenaKeyHolder{value, pool}, inserted);
|
||||
else
|
||||
set.emplace(DB::ArenaKeyHolder{value, pool}, it, inserted);
|
||||
}
|
||||
watch.stop();
|
||||
|
||||
std::cerr << "arena-memory " << pool.size() + set.getBufferSizeInBytes() << std::endl;
|
||||
std::cerr << "single-run " << std::setprecision(3)
|
||||
<< watch.elapsedSeconds() << std::endl;
|
||||
}
|
||||
}
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
{
|
||||
if (argc < 3)
|
||||
{
|
||||
std::cerr << "Usage: program n m\n";
|
||||
return 1;
|
||||
}
|
||||
|
||||
size_t n = std::stol(argv[1]);
|
||||
size_t m = std::stol(argv[2]);
|
||||
|
||||
DB::Arena pool(128 * 1024 * 1024);
|
||||
std::vector<StringRef> data(n);
|
||||
|
||||
std::cerr << "sizeof(Key) = " << sizeof(StringRef) << std::endl;
|
||||
|
||||
{
|
||||
Stopwatch watch;
|
||||
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
|
||||
DB::CompressedReadBuffer in2(in1);
|
||||
|
||||
std::string tmp;
|
||||
for (size_t i = 0; i < n && !in2.eof(); ++i)
|
||||
{
|
||||
DB::readStringBinary(tmp, in2);
|
||||
data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size());
|
||||
}
|
||||
|
||||
watch.stop();
|
||||
std::cerr << std::fixed << std::setprecision(2) << "Vector. Size: " << n << ", elapsed: " << watch.elapsedSeconds() << " ("
|
||||
<< n / watch.elapsedSeconds() << " elem/sec.)" << std::endl;
|
||||
}
|
||||
|
||||
if (!m || m == 1)
|
||||
bench<StringHashSet<>>(data, pool, "StringHashSet");
|
||||
if (!m || m == 2)
|
||||
bench<HashSetWithSavedHash<StringRef>>(data, pool, "HashSetWithSavedHash");
|
||||
if (!m || m == 3)
|
||||
bench<HashSet<StringRef>>(data, pool, "HashSet");
|
||||
return 0;
|
||||
}
|
@ -15,6 +15,16 @@ install (
|
||||
COMPONENT clickhouse
|
||||
PATTERN "CMakeLists.txt" EXCLUDE
|
||||
PATTERN ".gitignore" EXCLUDE
|
||||
PATTERN "top_level_domains" EXCLUDE
|
||||
)
|
||||
|
||||
# Dereference symlink
|
||||
get_filename_component(TOP_LEVEL_DOMAINS_ABS_DIR config/top_level_domains REALPATH)
|
||||
install (
|
||||
DIRECTORY "${TOP_LEVEL_DOMAINS_ABS_DIR}"
|
||||
DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test/config
|
||||
USE_SOURCE_PERMISSIONS
|
||||
COMPONENT clickhouse
|
||||
)
|
||||
|
||||
install (FILES server-test.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server COMPONENT clickhouse)
|
||||
|
5
tests/config/config.d/top_level_domains_lists.xml
Normal file
5
tests/config/config.d/top_level_domains_lists.xml
Normal file
@ -0,0 +1,5 @@
|
||||
<yandex>
|
||||
<top_level_domains_lists>
|
||||
<public_suffix_list>public_suffix_list.dat</public_suffix_list>
|
||||
</top_level_domains_lists>
|
||||
</yandex>
|
3
tests/config/config.d/top_level_domains_path.xml
Normal file
3
tests/config/config.d/top_level_domains_path.xml
Normal file
@ -0,0 +1,3 @@
|
||||
<yandex>
|
||||
<top_level_domains_path replace="replace">/etc/clickhouse-server/top_level_domains/</top_level_domains_path>
|
||||
</yandex>
|
@ -31,6 +31,8 @@ ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/c
|
||||
ln -sf $SRC_PATH/config.d/test_keeper_port.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/
|
||||
ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/
|
||||
ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/
|
||||
ln -sf $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/
|
||||
@ -42,6 +44,8 @@ ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/
|
||||
ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/
|
||||
ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/
|
||||
|
||||
ln -sf $SRC_PATH/top_level_domains $DEST_SERVER_PATH/
|
||||
|
||||
ln -sf $SRC_PATH/server.key $DEST_SERVER_PATH/
|
||||
ln -sf $SRC_PATH/server.crt $DEST_SERVER_PATH/
|
||||
ln -sf $SRC_PATH/dhparam.pem $DEST_SERVER_PATH/
|
||||
|
1
tests/config/top_level_domains
Symbolic link
1
tests/config/top_level_domains
Symbolic link
@ -0,0 +1 @@
|
||||
../../docker/test/performance-comparison/config/top_level_domains
|
@ -8,18 +8,6 @@
|
||||
<count>10</count>
|
||||
</logger>
|
||||
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<default>
|
||||
<type>s3</type>
|
||||
<endpoint>http://minio1:9001/root/data/</endpoint>
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
</default>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
|
||||
|
||||
<tcp_port>9000</tcp_port>
|
||||
<listen_host>127.0.0.1</listen_host>
|
||||
|
||||
|
@ -7,6 +7,7 @@
|
||||
<endpoint>http://minio1:9001/root/data/</endpoint>
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
<send_object_metadata>true</send_object_metadata>
|
||||
</default>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
|
@ -24,34 +24,38 @@ def cluster():
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def assert_objects_count(cluster, objects_count, path='data/'):
|
||||
minio = cluster.minio_client
|
||||
s3_objects = list(minio.list_objects(cluster.minio_bucket, path))
|
||||
print(s3_objects, file=sys.stderr)
|
||||
if objects_count != len(s3_objects):
|
||||
for s3_object in s3_objects:
|
||||
object_meta = minio.stat_object(cluster.minio_bucket, s3_object.object_name)
|
||||
logging.info("Existing S3 object: %s", str(object_meta))
|
||||
assert objects_count == len(s3_objects)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
"log_engine,files_overhead,files_overhead_per_insert",
|
||||
[("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)])
|
||||
def test_log_family_s3(cluster, log_engine, files_overhead, files_overhead_per_insert):
|
||||
node = cluster.instances["node"]
|
||||
minio = cluster.minio_client
|
||||
|
||||
node.query("CREATE TABLE s3_test (id UInt64) Engine={}".format(log_engine))
|
||||
|
||||
node.query("INSERT INTO s3_test SELECT number FROM numbers(5)")
|
||||
assert node.query("SELECT * FROM s3_test") == "0\n1\n2\n3\n4\n"
|
||||
print(list(minio.list_objects(cluster.minio_bucket, 'data/')), file=sys.stderr)
|
||||
assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == files_overhead_per_insert + files_overhead
|
||||
assert_objects_count(cluster, files_overhead_per_insert + files_overhead)
|
||||
|
||||
node.query("INSERT INTO s3_test SELECT number + 5 FROM numbers(3)")
|
||||
assert node.query("SELECT * FROM s3_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n"
|
||||
print(list(minio.list_objects(cluster.minio_bucket, 'data/')), file=sys.stderr)
|
||||
assert len(
|
||||
list(minio.list_objects(cluster.minio_bucket, 'data/'))) == files_overhead_per_insert * 2 + files_overhead
|
||||
assert_objects_count(cluster, files_overhead_per_insert * 2 + files_overhead)
|
||||
|
||||
node.query("INSERT INTO s3_test SELECT number + 8 FROM numbers(1)")
|
||||
assert node.query("SELECT * FROM s3_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n8\n"
|
||||
print(list(minio.list_objects(cluster.minio_bucket, 'data/')), file=sys.stderr)
|
||||
assert len(
|
||||
list(minio.list_objects(cluster.minio_bucket, 'data/'))) == files_overhead_per_insert * 3 + files_overhead
|
||||
assert_objects_count(cluster, files_overhead_per_insert * 3 + files_overhead)
|
||||
|
||||
node.query("TRUNCATE TABLE s3_test")
|
||||
print(list(minio.list_objects(cluster.minio_bucket, 'data/')), file=sys.stderr)
|
||||
assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0
|
||||
assert_objects_count(cluster, 0)
|
||||
|
||||
node.query("DROP TABLE s3_test")
|
||||
|
@ -1,14 +0,0 @@
|
||||
<test>
|
||||
|
||||
|
||||
|
||||
<preconditions>
|
||||
<table_exists>test.hits</table_exists>
|
||||
</preconditions>
|
||||
|
||||
<settings>
|
||||
<max_threads>1</max_threads>
|
||||
</settings>
|
||||
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(firstSignificantSubdomain(URL))</query>
|
||||
</test>
|
@ -1,11 +1,10 @@
|
||||
<test>
|
||||
|
||||
<preconditions>
|
||||
<table_exists>hits_100m_single</table_exists>
|
||||
<table_exists>test.hits</table_exists>
|
||||
</preconditions>
|
||||
|
||||
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>func</name>
|
||||
@ -32,6 +31,12 @@
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<query>SELECT count() FROM hits_100m_single WHERE NOT ignore({func}(URL))</query>
|
||||
|
||||
<!-- firstSignificantSubdomain/firstSignificantSubdomainCustom -->
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(firstSignificantSubdomain(URL)) SETTINGS max_threads=1</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(firstSignificantSubdomainCustom(URL, 'public_suffix_list')) SETTINGS max_threads=1</query>
|
||||
<!-- cutToFirstSignificantSubdomain/cutToFirstSignificantSubdomainCustom -->
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(cutToFirstSignificantSubdomain(URL)) SETTINGS max_threads=1</query>
|
||||
<query>SELECT count() FROM test.hits WHERE NOT ignore(cutToFirstSignificantSubdomainCustom(URL, 'public_suffix_list')) SETTINGS max_threads=1</query>
|
||||
</test>
|
||||
|
@ -0,0 +1,20 @@
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
1
|
||||
\N
|
||||
\N
|
||||
1
|
||||
\N
|
||||
\N
|
||||
0
|
||||
\N
|
||||
0
|
||||
\N
|
||||
1
|
||||
\N
|
||||
\N
|
||||
1
|
||||
\N
|
36
tests/queries/0_stateless/01562_agg_null_for_empty_ahead.sql
Normal file
36
tests/queries/0_stateless/01562_agg_null_for_empty_ahead.sql
Normal file
@ -0,0 +1,36 @@
|
||||
SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(0));
|
||||
SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(1));
|
||||
|
||||
SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(0)));
|
||||
SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(1)));
|
||||
|
||||
SELECT sumIf(1, 0);
|
||||
|
||||
SELECT sumIf(1, 1);
|
||||
|
||||
-- should return Null even if we donn't set aggregate_functions_null_for_empty
|
||||
SELECT sumIfOrNull(1, 0);
|
||||
SELECT sumOrNullIf(1, 0);
|
||||
|
||||
SELECT nullIf(1, 0);
|
||||
|
||||
SELECT nullIf(1, 1);
|
||||
|
||||
SET aggregate_functions_null_for_empty=1;
|
||||
|
||||
SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(0));
|
||||
SELECT sumMerge(s) FROM (SELECT sumState(number) s FROM numbers(1));
|
||||
|
||||
SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(0)));
|
||||
SELECT sumMerge(s) FROM (SELECT sumMergeState(n) s FROM (SELECT sumState(number) n FROM numbers(1)));
|
||||
|
||||
SELECT sumIf(1, 0);
|
||||
|
||||
SELECT sumIf(1, 1);
|
||||
|
||||
SELECT sumIfOrNull(1, 0);
|
||||
SELECT sumOrNullIf(1, 0);
|
||||
|
||||
SELECT nullIf(1, 0);
|
||||
|
||||
SELECT nullIf(1, 1);
|
11
tests/queries/0_stateless/01601_custom_tld.reference
Normal file
11
tests/queries/0_stateless/01601_custom_tld.reference
Normal file
@ -0,0 +1,11 @@
|
||||
no-tld
|
||||
|
||||
foo.there-is-no-such-domain
|
||||
foo.there-is-no-such-domain
|
||||
foo
|
||||
generic
|
||||
kernel
|
||||
kernel.biz.ss
|
||||
difference
|
||||
biz.ss
|
||||
kernel.biz.ss
|
16
tests/queries/0_stateless/01601_custom_tld.sql
Normal file
16
tests/queries/0_stateless/01601_custom_tld.sql
Normal file
@ -0,0 +1,16 @@
|
||||
select 'no-tld';
|
||||
select cutToFirstSignificantSubdomainCustom('there-is-no-such-domain', 'public_suffix_list');
|
||||
-- even if there is no TLD, 2-nd level by default anyway
|
||||
-- FIXME: make this behavior optional (so that TLD for host never changed, either empty or something real)
|
||||
select cutToFirstSignificantSubdomainCustom('foo.there-is-no-such-domain', 'public_suffix_list');
|
||||
select cutToFirstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list');
|
||||
select firstSignificantSubdomainCustom('bar.foo.there-is-no-such-domain', 'public_suffix_list');
|
||||
|
||||
select 'generic';
|
||||
select firstSignificantSubdomainCustom('foo.kernel.biz.ss', 'public_suffix_list'); -- kernel.biz.ss
|
||||
select cutToFirstSignificantSubdomainCustom('foo.kernel.biz.ss', 'public_suffix_list'); -- kernel.biz.ss
|
||||
|
||||
select 'difference';
|
||||
-- biz.ss is not in the default TLD list, hence:
|
||||
select cutToFirstSignificantSubdomain('foo.kernel.biz.ss'); -- biz.ss
|
||||
select cutToFirstSignificantSubdomainCustom('foo.kernel.biz.ss', 'public_suffix_list'); -- kernel.biz.ss
|
Loading…
Reference in New Issue
Block a user