Merge remote-tracking branch 'upstream/master' into materialized-postgresql-better-startup

This commit is contained in:
kssenii 2022-01-09 21:23:49 +03:00
commit 1984a80750
92 changed files with 2296 additions and 1189 deletions

2
contrib/NuRaft vendored

@ -1 +1 @@
Subproject commit ff100a8713146e1ca4b4158dd6cc4eef9af47fc3
Subproject commit c2043aa250e53ad5cf75e596e319d587af4dcb3c

View File

@ -7,7 +7,7 @@ toc_title: MaterializedPostgreSQL
Creates a ClickHouse database with tables from PostgreSQL database. Firstly, database with engine `MaterializedPostgreSQL` creates a snapshot of PostgreSQL database and loads required tables. Required tables can include any subset of tables from any subset of schemas from specified database. Along with the snapshot database engine acquires LSN and once initial dump of tables is performed - it starts pulling updates from WAL. After database is created, newly added tables to PostgreSQL database are not automatically added to replication. They have to be added manually with `ATTACH TABLE db.table` query.
Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. Such tables can be automatically reloaded in the background in case required setting is turned on. Safest way for now is to use `ATTACH`/ `DETACH` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position).
Replication is implemented with PostgreSQL Logical Replication Protocol, which does not allow to replicate DDL, but allows to know whether replication breaking changes happened (column type changes, adding/removing columns). Such changes are detected and according tables stop receiving updates. Such tables can be automatically reloaded in the background in case required setting is turned on (can be used starting from 22.1). Safest way for now is to use `ATTACH`/ `DETACH` queries to reload table completely. If DDL does not break replication (for example, renaming a column) table will still receive updates (insertion is done by position).
## Creating a Database {#creating-a-database}
@ -46,7 +46,7 @@ After `MaterializedPostgreSQL` database is created, it does not automatically de
ATTACH TABLE postgres_database.new_table;
```
Warning: before version 21.13 adding table to replication left unremoved temprorary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in clickhouse version before 21.13, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. Issue is fixed in 21.13.
Warning: before version 22.1 adding table to replication left unremoved temprorary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in clickhouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. Issue is fixed in 22.1.
## Dynamically removing tables from replication {#dynamically-removing-table-from-replication}
@ -156,6 +156,8 @@ Default value: empty list. (Default schema is used)
4. materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update}
Do not use this setting before 22.1 version.
Allows reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them.
Possible values:

View File

@ -1288,6 +1288,20 @@ Example
<mysql_port>9004</mysql_port>
```
## postgresql_port {#server_configuration_parameters-postgresql_port}
Port for communicating with clients over PostgreSQL protocol.
**Possible values**
Positive integer.
Example
``` xml
<postgresql_port>9005</postgresql_port>
```
## tmp_path {#tmp-path}
Path to temporary data for processing large queries.

View File

@ -1404,6 +1404,12 @@ Returns the index of the first element in the `arr1` array for which `func` retu
Note that the `arrayFirstIndex` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayLastIndex(func, arr1, …) {#array-last-index}
Returns the index of the last element in the `arr1` array for which `func` returns something other than 0.
Note that the `arrayLastIndex` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it cant be omitted.
## arrayMin {#array-min}
Returns the minimum of elements in the source array.

View File

@ -217,8 +217,8 @@ Result:
``` text
(0,'2019-05-20') 0 \N \N (NULL,NULL)
(1,'2019-05-20') 1 First First ('First','First')
(2,'2019-05-20') 0 \N \N (NULL,NULL)
(3,'2019-05-20') 0 \N \N (NULL,NULL)
(2,'2019-05-20') 1 Second \N ('Second',NULL)
(3,'2019-05-20') 1 Third Third ('Third','Third')
(4,'2019-05-20') 0 \N \N (NULL,NULL)
```

View File

@ -360,6 +360,21 @@ SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS Decod
└────────────────────────────────────────┘
```
### decodeURLFormComponent(URL) {#decodeurlformcomponenturl}
Returns the decoded URL. Follows rfc-1866, plain plus(`+`) is decoded as space(` `).
Example:
``` sql
SELECT decodeURLFormComponent('http://127.0.0.1:8123/?query=SELECT%201+2%2B3') AS DecodedURL;
```
``` text
┌─DecodedURL────────────────────────────────┐
│ http://127.0.0.1:8123/?query=SELECT 1 2+3 │
└───────────────────────────────────────────┘
```
### netloc {#netloc}
Extracts network locality (`username:password@host:port`) from a URL.

View File

@ -42,11 +42,14 @@ bool ACLMap::ACLsComparator::operator()(const Coordination::ACLs & left, const C
uint64_t ACLMap::convertACLs(const Coordination::ACLs & acls)
{
if (acls.empty())
return 0;
if (acl_to_num.count(acls))
return acl_to_num[acls];
/// Start from one
auto index = acl_to_num.size() + 1;
auto index = max_acl_id++;
acl_to_num[acls] = index;
num_to_acl[index] = acls;
@ -69,6 +72,7 @@ void ACLMap::addMapping(uint64_t acls_id, const Coordination::ACLs & acls)
{
num_to_acl[acls_id] = acls;
acl_to_num[acls] = acls_id;
max_acl_id = std::max(acls_id + 1, max_acl_id); /// max_acl_id pointer next slot
}
void ACLMap::addUsage(uint64_t acl_id)

View File

@ -31,6 +31,7 @@ private:
ACLToNumMap acl_to_num;
NumToACLMap num_to_acl;
UsageCounter usage_counter;
uint64_t max_acl_id{1};
public:
/// Convert ACL to number. If it's new ACL than adds it to map
@ -43,7 +44,7 @@ public:
/// Mapping from numbers to ACLs vectors. Used during serialization.
const NumToACLMap & getMapping() const { return num_to_acl; }
/// Add mapping to ACLMap. Used during deserialization.
/// Add mapping to ACLMap. Used during deserialization from snapshot.
void addMapping(uint64_t acls_id, const Coordination::ACLs & acls);
/// Add/remove usage of some id. Used to remove unused ACLs.

View File

@ -41,6 +41,7 @@ const String KeeperConfigurationAndSettings::DEFAULT_FOUR_LETTER_WORD_CMD = "con
KeeperConfigurationAndSettings::KeeperConfigurationAndSettings()
: server_id(NOT_EXIST)
, enable_ipv6(true)
, tcp_port(NOT_EXIST)
, tcp_port_secure(NOT_EXIST)
, standalone_keeper(false)
@ -67,6 +68,9 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const
writeText("server_id=", buf);
write_int(server_id);
writeText("enable_ipv6=", buf);
write_bool(enable_ipv6);
if (tcp_port != NOT_EXIST)
{
writeText("tcp_port=", buf);
@ -156,6 +160,8 @@ KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfigu
ret->server_id = config.getInt("keeper_server.server_id");
ret->standalone_keeper = standalone_keeper_;
ret->enable_ipv6 = config.getBool("keeper_server.enable_ipv6", true);
if (config.has("keeper_server.tcp_port"))
{
ret->tcp_port = config.getInt("keeper_server.tcp_port");

View File

@ -63,6 +63,7 @@ struct KeeperConfigurationAndSettings
KeeperConfigurationAndSettings();
int server_id;
bool enable_ipv6;
int tcp_port;
int tcp_port_secure;

View File

@ -276,7 +276,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
try
{
LOG_DEBUG(log, "Waiting server to initialize");
server->startup();
server->startup(configuration_and_settings->enable_ipv6);
LOG_DEBUG(log, "Server initialized, waiting for quorum");
if (!start_async)

View File

@ -107,7 +107,7 @@ KeeperServer::KeeperServer(
LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
}
void KeeperServer::startup()
void KeeperServer::startup(bool enable_ipv6)
{
state_machine->init();
@ -171,13 +171,14 @@ void KeeperServer::startup()
#endif
}
launchRaftServer(params, asio_opts);
launchRaftServer(enable_ipv6, params, asio_opts);
if (!raft_instance)
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
}
void KeeperServer::launchRaftServer(
bool enable_ipv6,
const nuraft::raft_params & params,
const nuraft::asio_service::options & asio_opts)
{
@ -192,7 +193,7 @@ void KeeperServer::launchRaftServer(
nuraft::ptr<nuraft::logger> logger = nuraft::cs_new<LoggerWrapper>("RaftInstance", coordination_settings->raft_logs_level);
asio_service = nuraft::cs_new<nuraft::asio_service>(asio_opts, logger);
asio_listener = asio_service->create_rpc_listener(state_manager->getPort(), logger);
asio_listener = asio_service->create_rpc_listener(state_manager->getPort(), logger, enable_ipv6);
if (!asio_listener)
return;

View File

@ -44,6 +44,7 @@ private:
/// Almost copy-paste from nuraft::launcher, but with separated server init and start
/// Allows to avoid race conditions.
void launchRaftServer(
bool enable_ipv6,
const nuraft::raft_params & params,
const nuraft::asio_service::options & asio_opts);
@ -57,7 +58,7 @@ public:
SnapshotsQueue & snapshots_queue_);
/// Load state machine from the latest snapshot and load log storage. Start NuRaft with required settings.
void startup();
void startup(bool enable_ipv6 = true);
/// Put local read request and execute in state machine directly and response into
/// responses queue

View File

@ -182,19 +182,19 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr,
return StoragePtr{};
auto connection_holder = pool->get();
auto columns = fetchPostgreSQLTableStructure(connection_holder->get(), table_name, configuration.schema).columns;
auto columns_info = fetchPostgreSQLTableStructure(connection_holder->get(), table_name, configuration.schema).physical_columns;
if (!columns)
if (!columns_info)
return StoragePtr{};
auto storage = StoragePostgreSQL::create(
StorageID(database_name, table_name), pool, table_name,
ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, configuration.schema, configuration.on_conflict);
ColumnsDescription{columns_info->columns}, ConstraintsDescription{}, String{}, configuration.schema, configuration.on_conflict);
if (cache_tables)
cached_tables[table_name] = storage;
return storage;
return std::move(storage);
}
if (table_checked || checkPostgresTable(table_name))
@ -414,7 +414,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, Co
assert(storage_engine_arguments->children.size() >= 2);
storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, std::make_shared<ASTLiteral>(table_id.table_name));
return create_table_query;
return std::move(create_table_query);
}

View File

@ -15,7 +15,7 @@
#include <Common/quoteString.h>
#include <Core/PostgreSQL/Utils.h>
#include <base/FnTraits.h>
#include <IO/ReadHelpers.h>
namespace DB
{
@ -155,10 +155,11 @@ static DataTypePtr convertPostgreSQLDataType(String & type, Fn<void()> auto && r
template<typename T>
std::shared_ptr<NamesAndTypesList> readNamesAndTypesList(
T & tx, const String & postgres_table, const String & query, bool use_nulls, bool only_names_and_types)
PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList(
T & tx, const String & postgres_table, const String & query, bool use_nulls, bool only_names_and_types)
{
auto columns = NamesAndTypes();
PostgreSQLTableStructure::Attributes attributes;
try
{
@ -180,14 +181,22 @@ std::shared_ptr<NamesAndTypesList> readNamesAndTypesList(
}
else
{
std::tuple<std::string, std::string, std::string, uint16_t> row;
std::tuple<std::string, std::string, std::string, uint16_t, std::string, std::string> row;
while (stream >> row)
{
auto data_type = convertPostgreSQLDataType(std::get<1>(row),
recheck_array,
use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false, i.e. value is nullable
std::get<3>(row));
auto data_type = convertPostgreSQLDataType(
std::get<1>(row), recheck_array,
use_nulls && (std::get<2>(row) == /* not nullable */"f"),
std::get<3>(row));
columns.push_back(NameAndTypePair(std::get<0>(row), data_type));
attributes.emplace_back(
PostgreSQLTableStructure::PGAttribute{
.atttypid = parse<int>(std::get<4>(row)),
.atttypmod = parse<int>(std::get<5>(row)),
});
++i;
}
}
@ -226,7 +235,9 @@ std::shared_ptr<NamesAndTypesList> readNamesAndTypesList(
throw;
}
return !columns.empty() ? std::make_shared<NamesAndTypesList>(columns.begin(), columns.end()) : nullptr;
return !columns.empty()
? std::make_shared<PostgreSQLTableStructure::ColumnsInfo>(NamesAndTypesList(columns.begin(), columns.end()), std::move(attributes))
: nullptr;
}
@ -244,14 +255,14 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure(
std::string query = fmt::format(
"SELECT attname AS name, format_type(atttypid, atttypmod) AS type, "
"attnotnull AS not_null, attndims AS dims "
"attnotnull AS not_null, attndims AS dims, atttypid as type_id, atttypmod as type_modifier "
"FROM pg_attribute "
"WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) "
"AND NOT attisdropped AND attnum > 0", where);
table.columns = readNamesAndTypesList(tx, postgres_table, query, use_nulls, false);
table.physical_columns = readNamesAndTypesList(tx, postgres_table, query, use_nulls, false);
if (!table.columns)
if (!table.physical_columns)
throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", postgres_table);
if (with_primary_key)

View File

@ -12,9 +12,24 @@ namespace DB
struct PostgreSQLTableStructure
{
std::shared_ptr<NamesAndTypesList> columns = nullptr;
std::shared_ptr<NamesAndTypesList> primary_key_columns = nullptr;
std::shared_ptr<NamesAndTypesList> replica_identity_columns = nullptr;
struct PGAttribute
{
Int32 atttypid;
Int32 atttypmod;
};
using Attributes = std::vector<PGAttribute>;
struct ColumnsInfo
{
NamesAndTypesList columns;
Attributes attributes;
ColumnsInfo(NamesAndTypesList && columns_, Attributes && attributes_) : columns(columns_), attributes(attributes_) {}
};
using ColumnsInfoPtr = std::shared_ptr<ColumnsInfo>;
ColumnsInfoPtr physical_columns;
ColumnsInfoPtr primary_key_columns;
ColumnsInfoPtr replica_identity_columns;
};
using PostgreSQLTableStructurePtr = std::unique_ptr<PostgreSQLTableStructure>;

View File

@ -13,6 +13,7 @@
#include <Dictionaries/ICacheDictionaryStorage.h>
#include <Dictionaries/DictionaryHelpers.h>
namespace DB
{
@ -308,7 +309,7 @@ private:
if (was_inserted)
{
if constexpr (std::is_same_v<KeyType, StringRef>)
cell.key = copyStringInArena(key);
cell.key = copyStringInArena(arena, key);
else
cell.key = key;
@ -332,8 +333,7 @@ private:
else if constexpr (std::is_same_v<ElementType, StringRef>)
{
const String & string_value = column_value.get<String>();
StringRef string_value_ref = StringRef {string_value.data(), string_value.size()};
StringRef inserted_value = copyStringInArena(string_value_ref);
StringRef inserted_value = copyStringInArena(arena, string_value);
container.back() = inserted_value;
}
else
@ -353,7 +353,7 @@ private:
{
char * data = const_cast<char *>(cell.key.data);
arena.free(data, cell.key.size);
cell.key = copyStringInArena(key);
cell.key = copyStringInArena(arena, key);
}
else
cell.key = key;
@ -379,8 +379,7 @@ private:
else if constexpr (std::is_same_v<ElementType, StringRef>)
{
const String & string_value = column_value.get<String>();
StringRef string_ref_value = StringRef {string_value.data(), string_value.size()};
StringRef inserted_value = copyStringInArena(string_ref_value);
StringRef inserted_value = copyStringInArena(arena, string_value);
if (!cell_was_default)
{
@ -423,7 +422,7 @@ private:
if (was_inserted)
{
if constexpr (std::is_same_v<KeyType, StringRef>)
cell.key = copyStringInArena(key);
cell.key = copyStringInArena(arena, key);
else
cell.key = key;
@ -463,7 +462,7 @@ private:
{
char * data = const_cast<char *>(cell.key.data);
arena.free(data, cell.key.size);
cell.key = copyStringInArena(key);
cell.key = copyStringInArena(arena, key);
}
else
cell.key = key;
@ -526,16 +525,6 @@ private:
return const_cast<std::decay_t<decltype(*this)> *>(this)->template getAttributeContainer(attribute_index, std::forward<GetContainerFunc>(func));
}
StringRef copyStringInArena(StringRef value_to_copy)
{
size_t value_to_copy_size = value_to_copy.size;
char * place_for_key = arena.alloc(value_to_copy_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(value_to_copy.data), value_to_copy_size);
StringRef updated_value{place_for_key, value_to_copy_size};
return updated_value;
}
template<typename ValueType>
using ContainerType = std::conditional_t<
std::is_same_v<ValueType, Field> || std::is_same_v<ValueType, Array>,

View File

@ -623,6 +623,17 @@ void mergeBlockWithPipe(
}
}
template <typename Arena>
static StringRef copyStringInArena(Arena & arena, StringRef value)
{
size_t key_size = value.size;
char * place_for_key = arena.alloc(key_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(value.data), key_size);
StringRef result{place_for_key, key_size};
return result;
}
/**
* Returns ColumnVector data as PaddedPodArray.

View File

@ -2,7 +2,6 @@
#include <Core/Defines.h>
#include <Common/HashTable/HashMap.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Functions/FunctionHelpers.h>
#include <Dictionaries/DictionaryFactory.h>

View File

@ -3,15 +3,12 @@
#include <atomic>
#include <variant>
#include <vector>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <Core/Block.h>
#include <Common/HashTable/HashMap.h>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryHelpers.h"
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryHelpers.h>
namespace DB
{

View File

@ -399,9 +399,6 @@ void FlatDictionary::calculateBytesAllocated()
}
bucket_count = container.capacity();
if constexpr (std::is_same_v<ValueType, StringRef>)
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
};
callOnDictionaryAttributeType(attribute.type, type_call);
@ -414,12 +411,14 @@ void FlatDictionary::calculateBytesAllocated()
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();
bytes_allocated += string_arena.size();
}
FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttribute & dictionary_attribute)
{
auto is_nullable_set = dictionary_attribute.is_nullable ? std::make_optional<NullableSet>() : std::optional<NullableSet>{};
Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), {}, {}};
Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), {}};
auto type_call = [&](const auto & dictionary_attribute_type)
{
@ -427,9 +426,6 @@ FlatDictionary::Attribute FlatDictionary::createAttribute(const DictionaryAttrib
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
if constexpr (std::is_same_v<ValueType, StringRef>)
attribute.string_arena = std::make_unique<Arena>();
attribute.container.emplace<ContainerType<ValueType>>(configuration.initial_array_size, ValueType());
};
@ -510,8 +506,8 @@ void FlatDictionary::setAttributeValueImpl(Attribute & attribute, UInt64 key, co
template <>
void FlatDictionary::setAttributeValueImpl<String>(Attribute & attribute, UInt64 key, const String & value)
{
const auto * string_in_arena = attribute.string_arena->insert(value.data(), value.size());
setAttributeValueImpl(attribute, key, StringRef{string_in_arena, value.size()});
auto arena_value = copyStringInArena(string_arena, value);
setAttributeValueImpl(attribute, key, arena_value);
}
void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, const Field & value)

View File

@ -133,8 +133,6 @@ private:
ContainerType<StringRef>,
ContainerType<Array>>
container;
std::unique_ptr<Arena> string_arena;
};
void createAttributes();
@ -176,6 +174,7 @@ private:
mutable std::atomic<size_t> found_count{0};
BlockPtr update_field_loaded_block;
Arena string_arena;
};
}

View File

@ -352,8 +352,7 @@ void HashedArrayDictionary<dictionary_key_type>::createAttributes()
using ValueType = DictionaryValueType<AttributeType>;
auto is_index_null = dictionary_attribute.is_nullable ? std::make_optional<std::vector<bool>>() : std::optional<std::vector<bool>>{};
std::unique_ptr<Arena> string_arena = std::is_same_v<AttributeType, String> ? std::make_unique<Arena>() : nullptr;
Attribute attribute{dictionary_attribute.underlying_type, AttributeContainerType<ValueType>(), std::move(is_index_null), std::move(string_arena)};
Attribute attribute{dictionary_attribute.underlying_type, AttributeContainerType<ValueType>(), std::move(is_index_null)};
attributes.emplace_back(std::move(attribute));
};
@ -431,7 +430,7 @@ void HashedArrayDictionary<dictionary_key_type>::blockToAttributes(const Block &
}
if constexpr (std::is_same_v<KeyType, StringRef>)
key = copyKeyInArena(key);
key = copyStringInArena(string_arena, key);
key_attribute.container.insert({key, element_count});
@ -466,11 +465,7 @@ void HashedArrayDictionary<dictionary_key_type>::blockToAttributes(const Block &
if constexpr (std::is_same_v<AttributeValueType, StringRef>)
{
String & value_to_insert = column_value_to_insert.get<String>();
size_t value_to_insert_size = value_to_insert.size();
const char * string_in_arena = attribute.string_arena->insert(value_to_insert.data(), value_to_insert_size);
StringRef string_in_arena_reference = StringRef{string_in_arena, value_to_insert_size};
StringRef string_in_arena_reference = copyStringInArena(string_arena, value_to_insert);
attribute_container.back() = string_in_arena_reference;
}
else
@ -676,16 +671,6 @@ void HashedArrayDictionary<dictionary_key_type>::getItemsImpl(
}
}
template <DictionaryKeyType dictionary_key_type>
StringRef HashedArrayDictionary<dictionary_key_type>::copyKeyInArena(StringRef key)
{
size_t key_size = key.size;
char * place_for_key = complex_key_arena.alloc(key_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(key.data), key_size);
StringRef updated_key{place_for_key, key_size};
return updated_key;
}
template <DictionaryKeyType dictionary_key_type>
void HashedArrayDictionary<dictionary_key_type>::loadData()
{
@ -742,21 +727,15 @@ void HashedArrayDictionary<dictionary_key_type>::calculateBytesAllocated()
}
bucket_count = container.capacity();
if constexpr (std::is_same_v<ValueType, StringRef>)
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
};
callOnDictionaryAttributeType(attribute.type, type_call);
if (attribute.string_arena)
bytes_allocated += attribute.string_arena->size();
if (attribute.is_index_null.has_value())
bytes_allocated += (*attribute.is_index_null).size();
}
bytes_allocated += complex_key_arena.size();
bytes_allocated += string_arena.size();
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();

View File

@ -155,7 +155,6 @@ private:
container;
std::optional<std::vector<bool>> is_index_null;
std::unique_ptr<Arena> string_arena;
};
struct KeyAttribute final
@ -205,8 +204,6 @@ private:
void resize(size_t added_rows);
StringRef copyKeyInArena(StringRef key);
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const HashedArrayDictionaryStorageConfiguration configuration;
@ -222,7 +219,7 @@ private:
mutable std::atomic<size_t> found_count{0};
BlockPtr update_field_loaded_block;
Arena complex_key_arena;
Arena string_arena;
};
extern template class HashedArrayDictionary<DictionaryKeyType::Simple>;

View File

@ -239,7 +239,7 @@ ColumnPtr HashedDictionary<dictionary_key_type, sparse>::getHierarchy(ColumnPtr
if (it != parent_keys_map.end())
result = getValueFromCell(it);
keys_found +=result.has_value();
keys_found += result.has_value();
return result;
};
@ -354,8 +354,7 @@ void HashedDictionary<dictionary_key_type, sparse>::createAttributes()
using ValueType = DictionaryValueType<AttributeType>;
auto is_nullable_set = dictionary_attribute.is_nullable ? std::make_optional<NullableSet>() : std::optional<NullableSet>{};
std::unique_ptr<Arena> string_arena = std::is_same_v<AttributeType, String> ? std::make_unique<Arena>() : nullptr;
Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), CollectionType<ValueType>(), std::move(string_arena)};
Attribute attribute{dictionary_attribute.underlying_type, std::move(is_nullable_set), CollectionType<ValueType>()};
attributes.emplace_back(std::move(attribute));
};
@ -449,7 +448,7 @@ void HashedDictionary<dictionary_key_type, sparse>::blockToAttributes(const Bloc
}
if constexpr (std::is_same_v<KeyType, StringRef>)
key = copyKeyInArena(key);
key = copyStringInArena(string_arena, key);
attribute_column.get(key_index, column_value_to_insert);
@ -463,12 +462,8 @@ void HashedDictionary<dictionary_key_type, sparse>::blockToAttributes(const Bloc
if constexpr (std::is_same_v<AttributeValueType, StringRef>)
{
String & value_to_insert = column_value_to_insert.get<String>();
size_t value_to_insert_size = value_to_insert.size();
const char * string_in_arena = attribute.string_arena->insert(value_to_insert.data(), value_to_insert_size);
StringRef string_in_arena_reference = StringRef{string_in_arena, value_to_insert_size};
container.insert({key, string_in_arena_reference});
StringRef arena_value = copyStringInArena(string_arena, value_to_insert);
container.insert({key, arena_value});
}
else
{
@ -548,16 +543,6 @@ void HashedDictionary<dictionary_key_type, sparse>::getItemsImpl(
found_count.fetch_add(keys_found, std::memory_order_relaxed);
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
StringRef HashedDictionary<dictionary_key_type, sparse>::copyKeyInArena(StringRef key)
{
size_t key_size = key.size;
char * place_for_key = complex_key_arena.alloc(key_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(key.data), key_size);
StringRef updated_key{place_for_key, key_size};
return updated_key;
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
void HashedDictionary<dictionary_key_type, sparse>::loadData()
{
@ -631,16 +616,13 @@ void HashedDictionary<dictionary_key_type, sparse>::calculateBytesAllocated()
}
});
if (attributes[i].string_arena)
bytes_allocated += attributes[i].string_arena->size();
bytes_allocated += sizeof(attributes[i].is_nullable_set);
if (attributes[i].is_nullable_set.has_value())
bytes_allocated = attributes[i].is_nullable_set->getBufferSizeInBytes();
}
bytes_allocated += complex_key_arena.size();
bytes_allocated += string_arena.size();
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();

View File

@ -173,8 +173,6 @@ private:
CollectionType<StringRef>,
CollectionType<Array>>
container;
std::unique_ptr<Arena> string_arena;
};
void createAttributes();
@ -202,8 +200,6 @@ private:
void resize(size_t added_rows);
StringRef copyKeyInArena(StringRef key);
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const HashedDictionaryStorageConfiguration configuration;
@ -217,7 +213,7 @@ private:
mutable std::atomic<size_t> found_count{0};
BlockPtr update_field_loaded_block;
Arena complex_key_arena;
Arena string_arena;
};
extern template class HashedDictionary<DictionaryKeyType::Simple, false>;

View File

@ -3,16 +3,14 @@
#include <atomic>
#include <variant>
#include <Core/Block.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/Arena.h>
#include <boost/geometry.hpp>
#include <boost/geometry/geometries/multi_polygon.hpp>
#include "DictionaryStructure.h"
#include "IDictionary.h"
#include "IDictionarySource.h"
#include "DictionaryHelpers.h"
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/DictionaryHelpers.h>
namespace DB
{

View File

@ -345,9 +345,6 @@ void RangeHashedDictionary<dictionary_key_type>::calculateBytesAllocated()
const auto & collection = std::get<CollectionType<ValueType>>(attribute.maps);
bytes_allocated += sizeof(CollectionType<ValueType>) + collection.getBufferSizeInBytes();
bucket_count = collection.getBufferSizeInCells();
if constexpr (std::is_same_v<ValueType, StringRef>)
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
};
callOnDictionaryAttributeType(attribute.type, type_call);
@ -358,12 +355,14 @@ void RangeHashedDictionary<dictionary_key_type>::calculateBytesAllocated()
if (update_field_loaded_block)
bytes_allocated += update_field_loaded_block->allocatedBytes();
bytes_allocated += string_arena.size();
}
template <DictionaryKeyType dictionary_key_type>
typename RangeHashedDictionary<dictionary_key_type>::Attribute RangeHashedDictionary<dictionary_key_type>::createAttribute(const DictionaryAttribute & dictionary_attribute)
{
Attribute attribute{dictionary_attribute.underlying_type, dictionary_attribute.is_nullable, {}, {}};
Attribute attribute{dictionary_attribute.underlying_type, dictionary_attribute.is_nullable, {}};
auto type_call = [&](const auto &dictionary_attribute_type)
{
@ -371,9 +370,6 @@ typename RangeHashedDictionary<dictionary_key_type>::Attribute RangeHashedDictio
using AttributeType = typename Type::AttributeType;
using ValueType = DictionaryValueType<AttributeType>;
if constexpr (std::is_same_v<AttributeType, String>)
attribute.string_arena = std::make_unique<Arena>();
attribute.maps = CollectionType<ValueType>();
};
@ -544,7 +540,7 @@ void RangeHashedDictionary<dictionary_key_type>::blockToAttributes(const Block &
}
if constexpr (std::is_same_v<KeyType, StringRef>)
key = copyKeyInArena(key);
key = copyStringInArena(string_arena, key);
setAttributeValue(attribute, key, Range{lower_bound, upper_bound}, attribute_column[key_index]);
keys_extractor.rollbackCurrentKey();
@ -572,8 +568,7 @@ void RangeHashedDictionary<dictionary_key_type>::setAttributeValueImpl(Attribute
if constexpr (std::is_same_v<T, String>)
{
const auto & string = value.get<String>();
const auto * string_in_arena = attribute.string_arena->insert(string.data(), string.size());
const StringRef string_ref{string_in_arena, string.size()};
StringRef string_ref = copyStringInArena(string_arena, string);
value_to_insert = Value<ValueType>{ range, { string_ref }};
}
else
@ -671,16 +666,6 @@ void RangeHashedDictionary<dictionary_key_type>::getKeysAndDates(
}
}
template <DictionaryKeyType dictionary_key_type>
StringRef RangeHashedDictionary<dictionary_key_type>::copyKeyInArena(StringRef key)
{
size_t key_size = key.size;
char * place_for_key = complex_key_arena.alloc(key_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(key.data), key_size);
StringRef updated_key{place_for_key, key_size};
return updated_key;
}
template <DictionaryKeyType dictionary_key_type>
template <typename RangeType>
PaddedPODArray<Int64> RangeHashedDictionary<dictionary_key_type>::makeDateKeys(

View File

@ -139,7 +139,6 @@ private:
CollectionType<StringRef>,
CollectionType<Array>>
maps;
std::unique_ptr<Arena> string_arena;
};
void createAttributes();
@ -162,9 +161,9 @@ private:
void blockToAttributes(const Block & block);
template <typename T>
static void setAttributeValueImpl(Attribute & attribute, KeyType key, const Range & range, const Field & value);
void setAttributeValueImpl(Attribute & attribute, KeyType key, const Range & range, const Field & value);
static void setAttributeValue(Attribute & attribute, KeyType key, const Range & range, const Field & value);
void setAttributeValue(Attribute & attribute, KeyType key, const Range & range, const Field & value);
template <typename RangeType>
void getKeysAndDates(
@ -184,8 +183,6 @@ private:
const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates) const;
StringRef copyKeyInArena(StringRef key);
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
@ -200,6 +197,7 @@ private:
size_t bucket_count = 0;
mutable std::atomic<size_t> query_count{0};
mutable std::atomic<size_t> found_count{0};
Arena string_arena;
};
}

View File

@ -1148,10 +1148,7 @@ private:
if constexpr (dictionary_key_type == DictionaryKeyType::Complex)
{
/// Copy complex key into arena and put in cache
size_t key_size = key.size;
char * place_for_key = complex_key_arena.alloc(key_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(key.data), key_size);
KeyType updated_key{place_for_key, key_size};
KeyType updated_key = copyStringInArena(complex_key_arena, key);
ssd_cache_key.key = updated_key;
}

View File

@ -35,7 +35,13 @@ namespace ErrorCodes
namespace
{
using NamesToTypeNames = std::unordered_map<std::string, std::string>;
struct AttributeConfiguration
{
std::string type;
std::string expression;
};
using AttributeNameToConfiguration = std::unordered_map<std::string, AttributeConfiguration>;
/// Get value from field and convert it to string.
/// Also remove quotes from strings.
@ -46,6 +52,21 @@ String getFieldAsString(const Field & field)
return applyVisitor(FieldVisitorToString(), field);
}
String getAttributeExpression(const ASTDictionaryAttributeDeclaration * dict_attr)
{
if (!dict_attr->expression)
return {};
/// EXPRESSION PROPERTY should be expression or string
String expression_str;
if (const auto * literal = dict_attr->expression->as<ASTLiteral>(); literal && literal->value.getType() == Field::Types::String)
expression_str = getFieldAsString(literal->value);
else
expression_str = queryToString(dict_attr->expression);
return expression_str;
}
using namespace Poco;
using namespace Poco::XML;
@ -63,20 +84,19 @@ void buildLifetimeConfiguration(
AutoPtr<Element> root,
const ASTDictionaryLifetime * lifetime)
{
if (!lifetime)
return;
if (lifetime)
{
AutoPtr<Element> lifetime_element(doc->createElement("lifetime"));
AutoPtr<Element> min_element(doc->createElement("min"));
AutoPtr<Element> max_element(doc->createElement("max"));
AutoPtr<Text> min_sec(doc->createTextNode(toString(lifetime->min_sec)));
min_element->appendChild(min_sec);
AutoPtr<Text> max_sec(doc->createTextNode(toString(lifetime->max_sec)));
max_element->appendChild(max_sec);
lifetime_element->appendChild(min_element);
lifetime_element->appendChild(max_element);
root->appendChild(lifetime_element);
}
AutoPtr<Element> lifetime_element(doc->createElement("lifetime"));
AutoPtr<Element> min_element(doc->createElement("min"));
AutoPtr<Element> max_element(doc->createElement("max"));
AutoPtr<Text> min_sec(doc->createTextNode(toString(lifetime->min_sec)));
min_element->appendChild(min_sec);
AutoPtr<Text> max_sec(doc->createTextNode(toString(lifetime->max_sec)));
max_element->appendChild(max_sec);
lifetime_element->appendChild(min_element);
lifetime_element->appendChild(max_element);
root->appendChild(lifetime_element);
}
/* Transforms next definition
@ -105,40 +125,43 @@ void buildLayoutConfiguration(
AutoPtr<Element> layout_type_element(doc->createElement(layout->layout_type));
layout_element->appendChild(layout_type_element);
if (layout->parameters)
if (!layout->parameters)
return;
for (const auto & param : layout->parameters->children)
{
for (const auto & param : layout->parameters->children)
const ASTPair * pair = param->as<ASTPair>();
if (!pair)
{
const ASTPair * pair = param->as<ASTPair>();
if (!pair)
{
throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary layout parameters must be key/value pairs, got '{}' instead",
param->formatForErrorMessage());
}
const ASTLiteral * value_literal = pair->second->as<ASTLiteral>();
if (!value_literal)
{
throw DB::Exception(ErrorCodes::BAD_ARGUMENTS,
"Dictionary layout parameter value must be a literal, got '{}' instead",
pair->second->formatForErrorMessage());
}
const auto value_field = value_literal->value;
if (value_field.getType() != Field::Types::UInt64
&& value_field.getType() != Field::Types::String)
{
throw DB::Exception(ErrorCodes::BAD_ARGUMENTS,
"Dictionary layout parameter value must be an UInt64 or String, got '{}' instead",
value_field.getTypeName());
}
AutoPtr<Element> layout_type_parameter_element(doc->createElement(pair->first));
AutoPtr<Text> value_to_append(doc->createTextNode(toString(value_field)));
layout_type_parameter_element->appendChild(value_to_append);
layout_type_element->appendChild(layout_type_parameter_element);
throw DB::Exception(
ErrorCodes::BAD_ARGUMENTS,
"Dictionary layout parameters must be key/value pairs, got '{}' instead",
param->formatForErrorMessage());
}
const ASTLiteral * value_literal = pair->second->as<ASTLiteral>();
if (!value_literal)
{
throw DB::Exception(
ErrorCodes::BAD_ARGUMENTS,
"Dictionary layout parameter value must be a literal, got '{}' instead",
pair->second->formatForErrorMessage());
}
const auto value_field = value_literal->value;
if (value_field.getType() != Field::Types::UInt64 && value_field.getType() != Field::Types::String)
{
throw DB::Exception(
ErrorCodes::BAD_ARGUMENTS,
"Dictionary layout parameter value must be an UInt64 or String, got '{}' instead",
value_field.getTypeName());
}
AutoPtr<Element> layout_type_parameter_element(doc->createElement(pair->first));
AutoPtr<Text> value_to_append(doc->createTextNode(toString(value_field)));
layout_type_parameter_element->appendChild(value_to_append);
layout_type_element->appendChild(layout_type_parameter_element);
}
}
@ -149,10 +172,10 @@ void buildLayoutConfiguration(
* <range_min><name>StartDate</name></range_min>
* <range_max><name>EndDate</name></range_max>
*/
void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTDictionaryRange * range, const NamesToTypeNames & all_attrs)
void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTDictionaryRange * range, const AttributeNameToConfiguration & all_attrs)
{
// appends <key><name>value</name></key> to root
auto append_element = [&doc, &root](const std::string & key, const std::string & name, const std::string & type)
auto append_element = [&doc, &root](const std::string & key, const std::string & name, const AttributeConfiguration & configuration)
{
AutoPtr<Element> element(doc->createElement(key));
AutoPtr<Element> name_node(doc->createElement("name"));
@ -161,22 +184,33 @@ void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const
element->appendChild(name_node);
AutoPtr<Element> type_node(doc->createElement("type"));
AutoPtr<Text> type_text(doc->createTextNode(type));
AutoPtr<Text> type_text(doc->createTextNode(configuration.type));
type_node->appendChild(type_text);
element->appendChild(type_node);
if (!configuration.expression.empty())
{
AutoPtr<Element> expression_node(doc->createElement("expression"));
AutoPtr<Text> expression_text(doc->createTextNode(configuration.expression));
expression_node->appendChild(expression_text);
element->appendChild(expression_node);
}
root->appendChild(element);
};
if (!all_attrs.count(range->min_attr_name))
auto range_min_attribute_it = all_attrs.find(range->min_attr_name);
if (range_min_attribute_it == all_attrs.end())
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"MIN ({}) attribute is not defined in the dictionary attributes", range->min_attr_name);
if (!all_attrs.count(range->max_attr_name))
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"MAX ({}) attribute is not defined in the dictionary attributes", range->max_attr_name);
"MIN {} attribute is not defined in the dictionary attributes", range->min_attr_name);
append_element("range_min", range->min_attr_name, all_attrs.at(range->min_attr_name));
append_element("range_max", range->max_attr_name, all_attrs.at(range->max_attr_name));
auto range_max_attribute_it = all_attrs.find(range->min_attr_name);
if (range_max_attribute_it == all_attrs.end())
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION,
"MAX {} attribute is not defined in the dictionary attributes", range->max_attr_name);
append_element("range_min", range->min_attr_name, range_min_attribute_it->second);
append_element("range_max", range->max_attr_name, range_max_attribute_it->second);
}
@ -199,25 +233,14 @@ void buildAttributeExpressionIfNeeded(
AutoPtr<Element> root,
const ASTDictionaryAttributeDeclaration * dict_attr)
{
if (dict_attr->expression != nullptr)
{
AutoPtr<Element> expression_element(doc->createElement("expression"));
if (!dict_attr->expression)
return;
/// EXPRESSION PROPERTY should be expression or string
String expression_str;
if (const auto * literal = dict_attr->expression->as<ASTLiteral>();
literal && literal->value.getType() == Field::Types::String)
{
expression_str = getFieldAsString(literal->value);
}
else
expression_str = queryToString(dict_attr->expression);
AutoPtr<Text> expression(doc->createTextNode(expression_str));
expression_element->appendChild(expression);
root->appendChild(expression_element);
}
AutoPtr<Element> expression_element(doc->createElement("expression"));
String expression_str = getAttributeExpression(dict_attr);
AutoPtr<Text> expression(doc->createTextNode(expression_str));
expression_element->appendChild(expression);
root->appendChild(expression_element);
}
/** Transofrms single dictionary attribute to configuration
@ -373,25 +396,28 @@ void buildPrimaryKeyConfiguration(
/** Transforms list of ASTDictionaryAttributeDeclarations to list of dictionary attributes
*/
NamesToTypeNames buildDictionaryAttributesConfiguration(
AttributeNameToConfiguration buildDictionaryAttributesConfiguration(
AutoPtr<Document> doc,
AutoPtr<Element> root,
const ASTExpressionList * dictionary_attributes,
const Names & key_columns)
{
const auto & children = dictionary_attributes->children;
NamesToTypeNames attributes_names_and_types;
AttributeNameToConfiguration attributes_name_to_configuration;
for (const auto & child : children)
{
const ASTDictionaryAttributeDeclaration * dict_attr = child->as<const ASTDictionaryAttributeDeclaration>();
if (!dict_attr->type)
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Dictionary attribute must has type");
attributes_names_and_types.emplace(dict_attr->name, queryToString(dict_attr->type));
AttributeConfiguration attribute_configuration {queryToString(dict_attr->type), getAttributeExpression(dict_attr)};
attributes_name_to_configuration.emplace(dict_attr->name, std::move(attribute_configuration));
if (std::find(key_columns.begin(), key_columns.end(), dict_attr->name) == key_columns.end())
buildSingleAttribute(doc, root, dict_attr);
}
return attributes_names_and_types;
return attributes_name_to_configuration;
}
/** Transform function with key-value arguments to configuration
@ -513,10 +539,10 @@ void checkAST(const ASTCreateQuery & query)
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Cannot create dictionary with empty source");
}
void checkPrimaryKey(const NamesToTypeNames & all_attrs, const Names & key_attrs)
void checkPrimaryKey(const AttributeNameToConfiguration & all_attrs, const Names & key_attrs)
{
for (const auto & key_attr : key_attrs)
if (all_attrs.count(key_attr) == 0)
if (all_attrs.find(key_attr) == all_attrs.end())
throw Exception(ErrorCodes::INCORRECT_DICTIONARY_DEFINITION, "Unknown key attribute '{}'", key_attr);
}

View File

@ -71,6 +71,20 @@ public:
void startup() override;
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override;
std::unique_ptr<ReadBufferFromFileBase> readMetaFile(
const String & path,
const ReadSettings & settings,
std::optional<size_t> size) const override { return delegate->readMetaFile(path, settings, size); }
std::unique_ptr<WriteBufferFromFileBase> writeMetaFile(
const String & path,
size_t buf_size,
WriteMode mode) override { return delegate->writeMetaFile(path, buf_size, mode); }
void removeMetaFileIfExists(const String & path) override { delegate->removeMetaFileIfExists(path); }
UInt32 getRefCount(const String & path) const override { return delegate->getRefCount(path); }
protected:
Executor & getExecutor() override;

View File

@ -86,4 +86,28 @@ SyncGuardPtr IDisk::getDirectorySyncGuard(const String & /* path */) const
return nullptr;
}
std::unique_ptr<ReadBufferFromFileBase> IDisk::readMetaFile(
const String & path,
const ReadSettings & settings,
std::optional<size_t> size) const
{
LOG_TRACE(&Poco::Logger::get("IDisk"), "Read local metafile: {}", path);
return readFile(path, settings, size);
}
std::unique_ptr<WriteBufferFromFileBase> IDisk::writeMetaFile(
const String & path,
size_t buf_size,
WriteMode mode)
{
LOG_TRACE(&Poco::Logger::get("IDisk"), "Write local metafile: {}", path);
return writeFile(path, buf_size, mode);
}
void IDisk::removeMetaFileIfExists(const String & path)
{
LOG_TRACE(&Poco::Logger::get("IDisk"), "Remove local metafile: {}", path);
removeFileIfExists(path);
}
}

View File

@ -248,6 +248,28 @@ public:
/// Applies new settings for disk in runtime.
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextPtr, const String &, const DisksMap &) {}
/// Open the local file for read and return ReadBufferFromFileBase object.
/// Overridden in IDiskRemote.
/// Used for work with custom metadata.
virtual std::unique_ptr<ReadBufferFromFileBase> readMetaFile(
const String & path,
const ReadSettings & settings,
std::optional<size_t> size) const;
/// Open the local file for write and return WriteBufferFromFileBase object.
/// Overridden in IDiskRemote.
/// Used for work with custom metadata.
virtual std::unique_ptr<WriteBufferFromFileBase> writeMetaFile(
const String & path,
size_t buf_size,
WriteMode mode);
virtual void removeMetaFileIfExists(const String & path);
/// Return reference count for remote FS.
/// Overridden in IDiskRemote.
virtual UInt32 getRefCount(const String &) const { return 0; }
protected:
friend class DiskDecorator;

View File

@ -484,6 +484,7 @@ bool IDiskRemote::tryReserve(UInt64 bytes)
String IDiskRemote::getUniqueId(const String & path) const
{
LOG_TRACE(log, "Remote path: {}, Path: {}", remote_fs_root_path, path);
Metadata metadata(remote_fs_root_path, metadata_disk, path);
String id;
if (!metadata.remote_fs_objects.empty())
@ -500,4 +501,34 @@ AsynchronousReaderPtr IDiskRemote::getThreadPoolReader()
return reader;
}
std::unique_ptr<ReadBufferFromFileBase> IDiskRemote::readMetaFile(
const String & path,
const ReadSettings & settings,
std::optional<size_t> size) const
{
LOG_TRACE(log, "Read metafile: {}", path);
return metadata_disk->readFile(path, settings, size);
}
std::unique_ptr<WriteBufferFromFileBase> IDiskRemote::writeMetaFile(
const String & path,
size_t buf_size,
WriteMode mode)
{
LOG_TRACE(log, "Write metafile: {}", path);
return metadata_disk->writeFile(path, buf_size, mode);
}
void IDiskRemote::removeMetaFileIfExists(const String & path)
{
LOG_TRACE(log, "Remove metafile: {}", path);
return metadata_disk->removeFileIfExists(path);
}
UInt32 IDiskRemote::getRefCount(const String & path) const
{
auto meta = readMeta(path);
return meta.ref_count;
}
}

View File

@ -136,6 +136,21 @@ public:
static AsynchronousReaderPtr getThreadPoolReader();
virtual std::unique_ptr<ReadBufferFromFileBase> readMetaFile(
const String & path,
const ReadSettings & settings,
std::optional<size_t> size) const override;
virtual std::unique_ptr<WriteBufferFromFileBase> writeMetaFile(
const String & path,
size_t buf_size,
WriteMode mode) override;
virtual void removeMetaFileIfExists(
const String & path) override;
UInt32 getRefCount(const String & path) const override;
protected:
Poco::Logger * log;
const String name;

View File

@ -12,7 +12,7 @@ namespace ErrorCodes
}
/// We assume that size of the dst buf isn't less than src_size.
static size_t decodeURL(const char * src, size_t src_size, char * dst)
static size_t decodeURL(const char * src, size_t src_size, char * dst, bool plus_as_space)
{
const char * src_prev_pos = src;
const char * src_curr_pos = src;
@ -21,12 +21,28 @@ static size_t decodeURL(const char * src, size_t src_size, char * dst)
while (true)
{
src_curr_pos = find_first_symbols<'%'>(src_curr_pos, src_end);
src_curr_pos = find_first_symbols<'%', '+'>(src_curr_pos, src_end);
if (src_curr_pos == src_end)
{
break;
}
else if (*src_curr_pos == '+')
{
if (!plus_as_space)
{
++src_curr_pos;
continue;
}
size_t bytes_to_copy = src_curr_pos - src_prev_pos;
memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy);
dst_pos += bytes_to_copy;
++src_curr_pos;
src_prev_pos = src_curr_pos;
*dst_pos = ' ';
++dst_pos;
}
else if (src_end - src_curr_pos < 3)
{
src_curr_pos = src_end;
@ -67,6 +83,7 @@ static size_t decodeURL(const char * src, size_t src_size, char * dst)
/// Percent decode of URL data.
template <bool plus_as_space>
struct DecodeURLComponentImpl
{
static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets,
@ -83,7 +100,7 @@ struct DecodeURLComponentImpl
{
const char * src_data = reinterpret_cast<const char *>(&data[prev_offset]);
size_t src_size = offsets[i] - prev_offset;
size_t dst_size = decodeURL(src_data, src_size, reinterpret_cast<char *>(res_data.data() + res_offset));
size_t dst_size = decodeURL(src_data, src_size, reinterpret_cast<char *>(res_data.data() + res_offset), plus_as_space);
res_offset += dst_size;
res_offsets[i] = res_offset;
@ -101,11 +118,14 @@ struct DecodeURLComponentImpl
struct NameDecodeURLComponent { static constexpr auto name = "decodeURLComponent"; };
using FunctionDecodeURLComponent = FunctionStringToString<DecodeURLComponentImpl, NameDecodeURLComponent>;
struct NameDecodeURLFormComponent { static constexpr auto name = "decodeURLFormComponent"; };
using FunctionDecodeURLComponent = FunctionStringToString<DecodeURLComponentImpl<false>, NameDecodeURLComponent>;
using FunctionDecodeURLFormComponent = FunctionStringToString<DecodeURLComponentImpl<true>, NameDecodeURLFormComponent>;
void registerFunctionDecodeURLComponent(FunctionFactory & factory)
{
factory.registerFunction<FunctionDecodeURLComponent>();
factory.registerFunction<FunctionDecodeURLFormComponent>();
}
}

View File

@ -1,90 +0,0 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include "FunctionArrayMapped.h"
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
struct ArrayFirstIndexImpl
{
static bool needBoolean() { return false; }
static bool needExpression() { return true; }
static bool needOneArray() { return false; }
static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypePtr & /*array_element*/)
{
return std::make_shared<DataTypeUInt32>();
}
static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped)
{
const auto * column_filter = typeid_cast<const ColumnUInt8 *>(&*mapped);
if (!column_filter)
{
const auto * column_filter_const = checkAndGetColumnConst<ColumnUInt8>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getValue<UInt8>())
{
const auto & offsets = array.getOffsets();
auto out_column = ColumnUInt32::create(offsets.size());
auto & out_index = out_column->getData();
size_t pos{};
for (size_t i = 0; i < offsets.size(); ++i)
{
out_index[i] = offsets[i] - pos > 0;
pos = offsets[i];
}
return out_column;
}
else
return DataTypeUInt32().createColumnConst(array.size(), 0u);
}
const auto & filter = column_filter->getData();
const auto & offsets = array.getOffsets();
auto out_column = ColumnUInt32::create(offsets.size());
auto & out_index = out_column->getData();
size_t pos{};
for (size_t i = 0; i < offsets.size(); ++i)
{
UInt32 index{};
for (size_t idx{1}; pos < offsets[i]; ++pos, ++idx)
{
if (filter[pos])
{
index = idx;
pos = offsets[i];
break;
}
}
out_index[i] = index;
}
return out_column;
}
};
struct NameArrayFirstIndex { static constexpr auto name = "arrayFirstIndex"; };
using FunctionArrayFirstIndex = FunctionArrayMapped<ArrayFirstIndexImpl, NameArrayFirstIndex>;
void registerFunctionArrayFirstIndex(FunctionFactory & factory)
{
factory.registerFunction<FunctionArrayFirstIndex>();
}
}

View File

@ -0,0 +1,134 @@
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include "FunctionArrayMapped.h"
#include <Functions/FunctionFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
enum class ArrayFirstLastIndexStrategy
{
First,
Last
};
template <ArrayFirstLastIndexStrategy strategy>
struct ArrayFirstLastIndexImpl
{
static bool needBoolean() { return false; }
static bool needExpression() { return true; }
static bool needOneArray() { return false; }
static DataTypePtr getReturnType(const DataTypePtr & /*expression_return*/, const DataTypePtr & /*array_element*/)
{
return std::make_shared<DataTypeUInt32>();
}
static ColumnPtr execute(const ColumnArray & array, ColumnPtr mapped)
{
const auto * column_filter = typeid_cast<const ColumnUInt8 *>(&*mapped);
if (!column_filter)
{
const auto * column_filter_const = checkAndGetColumnConst<ColumnUInt8>(&*mapped);
if (!column_filter_const)
throw Exception("Unexpected type of filter column", ErrorCodes::ILLEGAL_COLUMN);
if (column_filter_const->getValue<UInt8>())
{
const auto & offsets = array.getOffsets();
auto out_column = ColumnUInt32::create(offsets.size());
auto & out_index = out_column->getData();
size_t offsets_size = offsets.size();
for (size_t offset_index = 0; offset_index < offsets_size; ++offset_index)
{
size_t start_offset = offsets[offset_index - 1];
size_t end_offset = offsets[offset_index];
if (end_offset > start_offset)
{
if constexpr (strategy == ArrayFirstLastIndexStrategy::First)
out_index[offset_index] = 1;
else
out_index[offset_index] = end_offset - start_offset;
}
else
{
out_index[offset_index] = 0;
}
}
return out_column;
}
else
{
return DataTypeUInt32().createColumnConst(array.size(), 0u);
}
}
const auto & filter = column_filter->getData();
const auto & offsets = array.getOffsets();
size_t offsets_size = offsets.size();
auto out_column = ColumnUInt32::create(offsets_size);
auto & out_index = out_column->getData();
for (size_t offset_index = 0; offset_index < offsets_size; ++offset_index)
{
size_t start_offset = offsets[offset_index - 1];
size_t end_offset = offsets[offset_index];
size_t result_index = 0;
if constexpr (strategy == ArrayFirstLastIndexStrategy::First)
{
for (size_t index = 1; start_offset != end_offset; ++start_offset, ++index)
{
if (filter[start_offset])
{
result_index = index;
break;
}
}
}
else
{
for (size_t index = end_offset - start_offset; end_offset != start_offset; --end_offset, --index)
{
if (filter[end_offset - 1])
{
result_index = index;
break;
}
}
}
out_index[offset_index] = result_index;
}
return out_column;
}
};
struct NameArrayFirstIndex { static constexpr auto name = "arrayFirstIndex"; };
using ArrayFirstIndexImpl = ArrayFirstLastIndexImpl<ArrayFirstLastIndexStrategy::First>;
using FunctionArrayFirstIndex = FunctionArrayMapped<ArrayFirstIndexImpl, NameArrayFirstIndex>;
struct NameArrayLastIndex { static constexpr auto name = "arrayLastIndex"; };
using ArrayLastIndexImpl = ArrayFirstLastIndexImpl<ArrayFirstLastIndexStrategy::Last>;
using FunctionArrayLastIndex = FunctionArrayMapped<ArrayLastIndexImpl, NameArrayLastIndex>;
void registerFunctionArrayFirstIndex(FunctionFactory & factory)
{
factory.registerFunction<FunctionArrayFirstIndex>();
factory.registerFunction<FunctionArrayLastIndex>();
}
}

View File

@ -5,22 +5,25 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
class FunctionMonthNameImpl : public IFunction
class FunctionMonthName : public IFunction
{
public:
static constexpr auto name = "monthName";
static constexpr auto month_str = "month";
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionMonthNameImpl>(context); }
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionMonthName>(context); }
explicit FunctionMonthNameImpl(ContextPtr context_) : context(context_) {}
explicit FunctionMonthName(ContextPtr context_)
: function_resolver(FunctionFactory::instance().get("dateName", std::move(context_)))
{}
String getName() const override { return name; }
@ -61,17 +64,17 @@ public:
arguments[0]
};
auto date_name_func = FunctionFactory::instance().get("dateName", context)->build(temporary_columns);
auto date_name_func = function_resolver->build(temporary_columns);
return date_name_func->execute(temporary_columns, result_type, input_rows_count);
}
private:
ContextPtr context;
FunctionOverloadResolverPtr function_resolver;
};
void registerFunctionMonthName(FunctionFactory & factory)
{
factory.registerFunction<FunctionMonthNameImpl>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionMonthName>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -3,6 +3,7 @@
namespace DB
{
bool ParserKeyword::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::BareWord)
@ -36,4 +37,5 @@ bool ParserKeyword::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expecte
return true;
}
}

File diff suppressed because it is too large Load Diff

View File

@ -228,49 +228,6 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
ASTPtr createFunctionCast(const ASTPtr & expr_ast, const ASTPtr & type_ast);
class ParserCastAsExpression : public IParserBase
{
protected:
const char * getName() const override { return "CAST AS expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserSubstringExpression : public IParserBase
{
protected:
const char * getName() const override { return "SUBSTRING expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserTrimExpression : public IParserBase
{
protected:
const char * getName() const override { return "TRIM expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserExtractExpression : public IParserBase
{
protected:
const char * getName() const override { return "EXTRACT expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserDateAddExpression : public IParserBase
{
protected:
const char * getName() const override { return "DATE_ADD expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserDateDiffExpression : public IParserBase
{
protected:
const char * getName() const override { return "DATE_DIFF expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** NULL literal.
*/
class ParserNull : public IParserBase
@ -319,17 +276,6 @@ protected:
};
/**
* Parse query with EXISTS expression.
*/
class ParserExistsExpression : public IParserBase
{
protected:
const char * getName() const override { return "exists expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An array or tuple of literals.
* Arrays can also be parsed as an application of [] operator and tuples as an application of 'tuple' function.
* But parsing the whole array/tuple as a whole constant seriously speeds up the analysis of expressions in the case of very large collection.
@ -521,4 +467,6 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
ASTPtr createFunctionCast(const ASTPtr & expr_ast, const ASTPtr & type_ast);
}

View File

@ -689,7 +689,7 @@ bool ParserUnaryExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
bool ParserCastExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTPtr expr_ast;
if (!elem_parser.parse(pos, expr_ast, expected))
if (!ParserExpressionElement().parse(pos, expr_ast, expected))
return false;
ASTPtr type_ast;

View File

@ -203,9 +203,6 @@ protected:
/// Example: "[1, 1 + 1, 1 + 2]::Array(UInt8)"
class ParserCastExpression : public IParserBase
{
private:
ParserExpressionElement elem_parser;
protected:
const char * getName() const override { return "CAST expression"; }

View File

@ -60,7 +60,9 @@ public:
uint32_t depth = 0;
uint32_t max_depth = 0;
Pos(Tokens & tokens_, uint32_t max_depth_) : TokenIterator(tokens_), max_depth(max_depth_) {}
Pos(Tokens & tokens_, uint32_t max_depth_) : TokenIterator(tokens_), max_depth(max_depth_)
{
}
ALWAYS_INLINE void increaseDepth()
{

View File

@ -17,7 +17,7 @@ public:
Pos begin = pos;
bool res = func();
if (!res)
pos = begin;
pos = begin;
return res;
}
@ -31,7 +31,7 @@ public:
bool res = func();
pos.decreaseDepth();
if (!res)
pos = begin;
pos = begin;
return res;
}

View File

@ -289,8 +289,6 @@ void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const B
{
for (const auto & shard : shards)
{
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>();
if (shard.lazy)
addLazyPipe(pipes, shard, /*coordinator=*/nullptr, /*pool*/{}, /*replica_info*/std::nullopt);
else

View File

@ -325,7 +325,7 @@ void TCPHandler::runImpl()
if (state.is_cancelled)
return std::nullopt;
sendMergeTreeReadTaskRequstAssumeLocked(std::move(request));
sendMergeTreeReadTaskRequestAssumeLocked(std::move(request));
return receivePartitionMergeTreeReadTaskResponseAssumeLocked();
});
@ -805,7 +805,7 @@ void TCPHandler::sendReadTaskRequestAssumeLocked()
}
void TCPHandler::sendMergeTreeReadTaskRequstAssumeLocked(PartitionReadRequest request)
void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(PartitionReadRequest request)
{
writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out);
request.serialize(*out);

View File

@ -239,7 +239,7 @@ private:
void sendEndOfStream();
void sendPartUUIDs();
void sendReadTaskRequestAssumeLocked();
void sendMergeTreeReadTaskRequstAssumeLocked(PartitionReadRequest request);
void sendMergeTreeReadTaskRequestAssumeLocked(PartitionReadRequest request);
void sendProfileInfo(const ProfileInfo & info);
void sendTotals(const Block & totals);
void sendExtremes(const Block & extremes);

View File

@ -1159,6 +1159,14 @@ void IMergeTreeDataPart::renameTo(const String & new_relative_path, bool remove_
storage.lockSharedData(*this);
}
void IMergeTreeDataPart::cleanupOldName(const String & old_part_name) const
{
if (name == old_part_name)
return;
storage.unlockSharedData(*this, old_part_name);
}
std::optional<bool> IMergeTreeDataPart::keepSharedDataInDecoupledStorage() const
{
/// NOTE: It's needed for zero-copy replication
@ -1621,6 +1629,12 @@ String IMergeTreeDataPart::getUniqueId() const
}
UInt32 IMergeTreeDataPart::getNumberOfRefereneces() const
{
return volume->getDisk()->getRefCount(fs::path(getFullRelativePath()) / "checksums.txt");
}
String IMergeTreeDataPart::getZeroLevelPartBlockID() const
{
if (info.level != 0)

View File

@ -338,6 +338,9 @@ public:
/// Changes only relative_dir_name, you need to update other metadata (name, is_temp) explicitly
virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const;
/// Cleanup shared locks made with old name after part renaming
virtual void cleanupOldName(const String & old_part_name) const;
/// Makes clone of a part in detached/ directory via hard links
virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const;
@ -404,10 +407,14 @@ public:
/// part creation (using alter query with materialize_ttl setting).
bool checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const;
/// Return some uniq string for file
/// Required for distinguish different copies of the same part on S3
/// Return some uniq string for file.
/// Required for distinguish different copies of the same part on remote FS.
String getUniqueId() const;
/// Return hardlink count for part.
/// Required for keep data on remote FS when part has shadow copies.
UInt32 getNumberOfRefereneces() const;
protected:
/// Total size of all columns, calculated once in calcuateColumnSizesOnDisk

View File

@ -22,6 +22,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER;
extern const int LOGICAL_ERROR;
extern const int QUERY_WAS_CANCELLED;
}
@ -131,8 +132,9 @@ bool MergeTreeBaseSelectProcessor::getTaskFromBuffer()
if (Status::Accepted == res)
return true;
/// To avoid any possibility of ignoring cancellation, exception will be thrown.
if (Status::Cancelled == res)
break;
throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Query had been cancelled");
}
return false;
}
@ -165,8 +167,18 @@ Chunk MergeTreeBaseSelectProcessor::generate()
{
while (!isCancelled())
{
if ((!task || task->isFinished()) && !getNewTask())
return {};
try
{
if ((!task || task->isFinished()) && !getNewTask())
return {};
}
catch (const Exception & e)
{
/// See MergeTreeBaseSelectProcessor::getTaskFromBuffer()
if (e.code() == ErrorCodes::QUERY_WAS_CANCELLED)
return {};
throw;
}
auto res = readFromPart();

View File

@ -63,6 +63,7 @@
#include <boost/range/adaptor/filtered.hpp>
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <base/insertAtEnd.h>
#include <base/scope_guard_safe.h>
@ -2458,6 +2459,8 @@ bool MergeTreeData::renameTempPartAndReplace(
MergeTreePartInfo part_info = part->info;
String part_name;
String old_part_name = part->name;
if (DataPartPtr existing_part_in_partition = getAnyPartInPartition(part->info.partition_id, lock))
{
if (part->partition.value != existing_part_in_partition->partition.value)
@ -2521,6 +2524,7 @@ bool MergeTreeData::renameTempPartAndReplace(
/// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts
///
/// If out_transaction is null, we commit the part to the active set immediately, else add it to the transaction.
part->name = part_name;
part->info = part_info;
part->is_temp = false;
@ -2569,6 +2573,9 @@ bool MergeTreeData::renameTempPartAndReplace(
out_covered_parts->emplace_back(std::move(covered_part));
}
/// Cleanup shared locks made with old name
part->cleanupOldName(old_part_name);
return true;
}
@ -3907,8 +3914,8 @@ void MergeTreeData::dropDetached(const ASTPtr & partition, bool part, ContextPtr
for (auto & [old_name, new_name, disk] : renamed_parts.old_and_new_names)
{
disk->removeRecursive(fs::path(relative_data_path) / "detached" / new_name / "");
LOG_DEBUG(log, "Dropped detached part {}", old_name);
bool keep_shared = removeDetachedPart(disk, fs::path(relative_data_path) / "detached" / new_name / "", old_name, false);
LOG_DEBUG(log, "Dropped detached part {}, keep shared data: {}", old_name, keep_shared);
old_name.clear();
}
}
@ -5196,7 +5203,9 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
LOG_DEBUG(log, "Freezing part {} snapshot will be placed at {}", part->name, backup_path);
part->volume->getDisk()->createDirectories(backup_path);
auto disk = part->volume->getDisk();
disk->createDirectories(backup_path);
String src_part_path = part->getFullRelativePath();
String backup_part_path = fs::path(backup_path) / relative_data_path / part->relative_path;
@ -5207,16 +5216,20 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
src_part_path = fs::path(relative_data_path) / flushed_part_path / "";
}
localBackup(part->volume->getDisk(), src_part_path, backup_part_path);
localBackup(disk, src_part_path, backup_part_path);
part->volume->getDisk()->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME);
// Store metadata for replicated table.
// Do nothing for non-replocated.
createAndStoreFreezeMetadata(disk, part, backup_part_path);
disk->removeFileIfExists(fs::path(backup_part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME);
part->is_frozen.store(true, std::memory_order_relaxed);
result.push_back(PartitionCommandResultInfo{
.partition_id = part->info.partition_id,
.part_name = part->name,
.backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_path,
.part_backup_path = fs::path(part->volume->getDisk()->getPath()) / backup_part_path,
.backup_path = fs::path(disk->getPath()) / backup_path,
.part_backup_path = fs::path(disk->getPath()) / backup_part_path,
.backup_name = backup_name,
});
++parts_processed;
@ -5226,6 +5239,11 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
return result;
}
void MergeTreeData::createAndStoreFreezeMetadata(DiskPtr, DataPartPtr, String) const
{
}
PartitionCommandsResultInfo MergeTreeData::unfreezePartition(
const ASTPtr & partition,
const String & backup_name,
@ -5243,6 +5261,13 @@ PartitionCommandsResultInfo MergeTreeData::unfreezeAll(
return unfreezePartitionsByMatcher([] (const String &) { return true; }, backup_name, local_context);
}
bool MergeTreeData::removeDetachedPart(DiskPtr disk, const String & path, const String &, bool)
{
disk->removeRecursive(path);
return false;
}
PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn matcher, const String & backup_name, ContextPtr)
{
auto backup_path = fs::path("shadow") / escapeForFileName(backup_name) / relative_data_path;
@ -5271,7 +5296,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn
const auto & path = it->path();
disk->removeRecursive(path);
bool keep_shared = removeDetachedPart(disk, path, partition_directory, true);
result.push_back(PartitionCommandResultInfo{
.partition_id = partition_id,
@ -5281,7 +5306,7 @@ PartitionCommandsResultInfo MergeTreeData::unfreezePartitionsByMatcher(MatcherFn
.backup_name = backup_name,
});
LOG_DEBUG(log, "Unfreezed part by path {}", disk->getPath() + path);
LOG_DEBUG(log, "Unfreezed part by path {}, keep shared data: {}", disk->getPath() + path, keep_shared);
}
}

View File

@ -873,10 +873,21 @@ public:
/// Overridden in StorageReplicatedMergeTree
virtual bool unlockSharedData(const IMergeTreeDataPart &) const { return true; }
/// Remove lock with old name for shared data part after rename
virtual bool unlockSharedData(const IMergeTreeDataPart &, const String &) const { return true; }
/// Fetch part only if some replica has it on shared storage like S3
/// Overridden in StorageReplicatedMergeTree
virtual bool tryToFetchIfShared(const IMergeTreeDataPart &, const DiskPtr &, const String &) { return false; }
/// Check shared data usage on other replicas for detached/freezed part
/// Remove local files and remote files if needed
virtual bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed);
/// Store metadata for replicated tables
/// Do nothing for non-replicated tables
virtual void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const;
/// Parts that currently submerging (merging to bigger parts) or emerging
/// (to be appeared after merging finished). These two variables have to be used
/// with `currently_submerging_emerging_mutex`.

View File

@ -125,8 +125,10 @@ struct Settings;
M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \
M(String, storage_policy, "default", "Name of storage disk policy", 0) \
M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \
M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs", 0) \
M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \
M(Bool, allow_remote_fs_zero_copy_replication, true, "Allow Zero-copy replication over remote fs.", 0) \
M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for Zero-copy table-independet info.", 0) \
M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \
M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \
M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \
M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \
M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \

View File

@ -228,6 +228,8 @@ void ReplicatedMergeTreeSink::commitPart(
bool is_already_existing_part = false;
String old_part_name = part->name;
while (true)
{
/// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem.
@ -508,6 +510,9 @@ void ReplicatedMergeTreeSink::commitPart(
waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value);
}
/// Cleanup shared locks made with old name
part->cleanupOldName(old_part_name);
}
void ReplicatedMergeTreeSink::onStart()

View File

@ -18,6 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int POSTGRESQL_REPLICATION_INTERNAL_ERROR;
}
MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer(
@ -29,7 +30,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer(
const size_t max_block_size_,
bool schema_as_a_part_of_table_name_,
bool allow_automatic_update_,
Storages storages_,
StorageInfos storages_info_,
const String & name_for_logger)
: log(&Poco::Logger::get("PostgreSQLReplicaConsumer(" + name_for_logger + ")"))
, context(context_)
@ -41,7 +42,6 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer(
, max_block_size(max_block_size_)
, schema_as_a_part_of_table_name(schema_as_a_part_of_table_name_)
, allow_automatic_update(allow_automatic_update_)
, storages(storages_)
{
final_lsn = start_lsn;
auto tx = std::make_shared<pqxx::nontransaction>(connection->getRef());
@ -49,19 +49,28 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer(
LOG_TRACE(log, "Starting replication. LSN: {} (last: {})", getLSNValue(current_lsn), getLSNValue(final_lsn));
tx->commit();
for (const auto & [table_name, storage] : storages)
{
buffers.emplace(table_name, Buffer(storage));
}
for (const auto & [table_name, storage_info] : storages_info_)
storages.emplace(table_name, storage_info);
}
void MaterializedPostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage)
MaterializedPostgreSQLConsumer::StorageData::StorageData(const StorageInfo & storage_info)
: storage(storage_info.storage), buffer(storage_info.storage->getInMemoryMetadataPtr(), storage_info.attributes)
{
auto table_id = storage_info.storage->getStorageID();
LOG_TRACE(&Poco::Logger::get("StorageMaterializedPostgreSQL"),
"New buffer for table {}, number of attributes: {}, number if columns: {}, structure: {}",
table_id.getNameForLogs(), buffer.attributes.size(), buffer.getColumnsNum(), buffer.description.sample_block.dumpStructure());
}
MaterializedPostgreSQLConsumer::StorageData::Buffer::Buffer(
StorageMetadataPtr storage_metadata, const PostgreSQLTableStructure::Attributes & attributes_)
: attributes(attributes_)
{
const auto storage_metadata = storage->getInMemoryMetadataPtr();
const Block sample_block = storage_metadata->getSampleBlock();
/// Need to clear type, because in description.init() the types are appended (emplace_back)
/// Need to clear type, because in description.init() the types are appended
description.types.clear();
description.init(sample_block);
@ -69,13 +78,13 @@ void MaterializedPostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storag
const auto & storage_columns = storage_metadata->getColumns().getAllPhysical();
auto insert_columns = std::make_shared<ASTExpressionList>();
auto table_id = storage->getStorageID();
LOG_TRACE(&Poco::Logger::get("MaterializedPostgreSQLBuffer"), "New buffer for table {}.{} ({}), structure: {}",
table_id.database_name, table_id.table_name, toString(table_id.uuid), sample_block.dumpStructure());
auto columns_num = description.sample_block.columns();
assert(columns_num == storage_columns.size());
if (attributes.size() + 2 != columns_num) /// +2 because sign and version columns
throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns number mismatch. Attributes: {}, buffer: {}",
attributes.size(), columns_num);
assert(description.sample_block.columns() == storage_columns.size());
size_t idx = 0;
for (const auto & column : storage_columns)
{
if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray)
@ -85,37 +94,45 @@ void MaterializedPostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storag
insert_columns->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
}
columnsAST = std::move(insert_columns);
columns_ast = std::move(insert_columns);
}
void MaterializedPostgreSQLConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx)
void MaterializedPostgreSQLConsumer::insertValue(StorageData::Buffer & buffer, const std::string & value, size_t column_idx)
{
const auto & sample = buffer.description.sample_block.getByPosition(column_idx);
bool is_nullable = buffer.description.types[column_idx].second;
if (is_nullable)
try
{
ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(*buffer.columns[column_idx]);
const auto & data_type = assert_cast<const DataTypeNullable &>(*sample.type);
if (is_nullable)
{
ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(*buffer.columns[column_idx]);
const auto & data_type = assert_cast<const DataTypeNullable &>(*sample.type);
insertPostgreSQLValue(
column_nullable.getNestedColumn(), value,
buffer.description.types[column_idx].first, data_type.getNestedType(), buffer.array_info, column_idx);
insertPostgreSQLValue(
column_nullable.getNestedColumn(), value,
buffer.description.types[column_idx].first, data_type.getNestedType(), buffer.array_info, column_idx);
column_nullable.getNullMapData().emplace_back(0);
column_nullable.getNullMapData().emplace_back(0);
}
else
{
insertPostgreSQLValue(
*buffer.columns[column_idx], value,
buffer.description.types[column_idx].first, sample.type,
buffer.array_info, column_idx);
}
}
else
catch (const pqxx::conversion_error & e)
{
insertPostgreSQLValue(
*buffer.columns[column_idx], value,
buffer.description.types[column_idx].first, sample.type,
buffer.array_info, column_idx);
LOG_ERROR(log, "Conversion failed while inserting PostgreSQL value {}, will insert default value. Error: {}", value, e.what());
insertDefaultValue(buffer, column_idx);
}
}
void MaterializedPostgreSQLConsumer::insertDefaultValue(Buffer & buffer, size_t column_idx)
void MaterializedPostgreSQLConsumer::insertDefaultValue(StorageData::Buffer & buffer, size_t column_idx)
{
const auto & sample = buffer.description.sample_block.getByPosition(column_idx);
insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column);
@ -186,10 +203,16 @@ Int8 MaterializedPostgreSQLConsumer::readInt8(const char * message, size_t & pos
void MaterializedPostgreSQLConsumer::readTupleData(
Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value)
StorageData::Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value)
{
Int16 num_columns = readInt16(message, pos, size);
/// Sanity check. In fact, it was already checked.
if (static_cast<size_t>(num_columns) + 2 != buffer.getColumnsNum()) /// +2 -- sign and version columns
throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR,
"Number of columns does not match. Got: {}, expected {}, current buffer structure: {}",
num_columns, buffer.getColumnsNum(), buffer.description.sample_block.dumpStructure());
auto proccess_column_value = [&](Int8 identifier, Int16 column_idx)
{
switch (identifier)
@ -202,8 +225,15 @@ void MaterializedPostgreSQLConsumer::readTupleData(
case 't': /// Text formatted value
{
Int32 col_len = readInt32(message, pos, size);
String value;
/// Sanity check for protocol misuse.
/// PostgreSQL uses a fixed page size (commonly 8 kB), and does not allow tuples to span multiple pages.
static constexpr Int32 sanity_check_max_col_len = 1024 * 8 * 2; /// *2 -- just in case.
if (unlikely(col_len > sanity_check_max_col_len))
throw Exception(ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR,
"Column length is suspiciously long: {}", col_len);
String value;
for (Int32 i = 0; i < col_len; ++i)
value += readInt8(message, pos, size);
@ -276,19 +306,20 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
{
Int32 relation_id = readInt32(replication_message, pos, size);
const auto & table_name = relation_id_to_name[relation_id];
/// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove?
if (table_name.empty())
LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id);
{
LOG_ERROR(log, "No table mapping for relation id: {}. It's a bug", relation_id);
return;
}
if (!isSyncAllowed(relation_id, table_name))
return;
Int8 new_tuple = readInt8(replication_message, pos, size);
auto buffer = buffers.find(table_name);
assert(buffer != buffers.end());
auto & buffer = storages.find(table_name)->second.buffer;
if (new_tuple)
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::INSERT);
readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::INSERT);
break;
}
@ -296,15 +327,16 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
{
Int32 relation_id = readInt32(replication_message, pos, size);
const auto & table_name = relation_id_to_name[relation_id];
/// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove?
if (table_name.empty())
LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id);
{
LOG_ERROR(log, "No table mapping for relation id: {}. It's a bug", relation_id);
return;
}
if (!isSyncAllowed(relation_id, table_name))
return;
auto buffer = buffers.find(table_name);
assert(buffer != buffers.end());
auto & buffer = storages.find(table_name)->second.buffer;
auto proccess_identifier = [&](Int8 identifier) -> bool
{
@ -319,13 +351,13 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
/// it is much more efficient to use replica identity index, but support all possible cases.
case 'O':
{
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true);
readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::UPDATE, true);
break;
}
case 'N':
{
/// New row.
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE);
readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::UPDATE);
read_next = false;
break;
}
@ -347,9 +379,11 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
{
Int32 relation_id = readInt32(replication_message, pos, size);
const auto & table_name = relation_id_to_name[relation_id];
/// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove?
if (table_name.empty())
LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id);
{
LOG_ERROR(log, "No table mapping for relation id: {}. It's a bug", relation_id);
return;
}
if (!isSyncAllowed(relation_id, table_name))
return;
@ -357,10 +391,8 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
/// 0 or 1 if replica identity is set to full. For now only default replica identity is supported (with primary keys).
readInt8(replication_message, pos, size);
auto buffer = buffers.find(table_name);
assert(buffer != buffers.end());
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE);
auto & buffer = storages.find(table_name)->second.buffer;
readTupleData(buffer, replication_message, pos, size, PostgreSQLQuery::DELETE);
break;
}
case 'C': // Commit
@ -379,7 +411,6 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
Int32 relation_id = readInt32(replication_message, pos, size);
String relation_namespace, relation_name;
readString(replication_message, pos, size, relation_namespace);
readString(replication_message, pos, size, relation_name);
@ -389,22 +420,26 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
else
table_name = relation_name;
if (!relation_id_to_name.contains(relation_id))
relation_id_to_name[relation_id] = table_name;
if (!isSyncAllowed(relation_id, relation_name))
return;
if (storages.find(table_name) == storages.end())
auto storage_iter = storages.find(table_name);
if (storage_iter == storages.end())
{
markTableAsSkipped(relation_id, table_name);
/// TODO: This can happen if we created a publication with this table but then got an exception that this
/// FIXME: This can happen if we created a publication with this table but then got an exception that this
/// table has primary key or something else.
LOG_ERROR(log,
"Storage for table {} does not exist, but is included in replication stream. (Storages number: {})",
"Storage for table {} does not exist, but is included in replication stream. (Storages number: {})"
"Please manually remove this table from replication (DETACH TABLE query) to avoid redundant replication",
table_name, storages.size());
markTableAsSkipped(relation_id, table_name);
return;
}
assert(buffers.contains(table_name));
auto & buffer = storage_iter->second.buffer;
/// 'd' - default (primary key if any)
/// 'n' - nothing
@ -412,7 +447,6 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
/// 'i' - user defined index with indisreplident set
/// Only 'd' and 'i' - are supported.
char replica_identity = readInt8(replication_message, pos, size);
if (replica_identity != 'd' && replica_identity != 'i')
{
LOG_WARNING(log,
@ -423,25 +457,29 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
Int16 num_columns = readInt16(replication_message, pos, size);
Int32 data_type_id;
Int32 type_modifier; /// For example, n in varchar(n)
bool new_relation_definition = false;
if (schema_data.find(relation_id) == schema_data.end())
{
relation_id_to_name[relation_id] = table_name;
schema_data.emplace(relation_id, SchemaData(num_columns));
new_relation_definition = true;
}
auto & current_schema_data = schema_data.find(relation_id)->second;
if (current_schema_data.number_of_columns != num_columns)
if (static_cast<size_t>(num_columns) + 2 != buffer.getColumnsNum()) /// +2 -- sign and version columns
{
markTableAsSkipped(relation_id, table_name);
return;
}
if (static_cast<size_t>(num_columns) != buffer.attributes.size())
{
#ifndef NDEBUG
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Mismatch in attributes size. Got {}, expected {}. It's a bug. Current buffer structure: {}",
num_columns, buffer.attributes.size(), buffer.description.sample_block.dumpStructure());
#else
LOG_ERROR(log, "Mismatch in attributes size. Got {}, expected {}. It's a bug. Current buffer structure: {}",
num_columns, buffer.attributes.size(), buffer.description.sample_block.dumpStructure());
markTableAsSkipped(relation_id, table_name);
return;
#endif
}
Int32 data_type_id;
Int32 type_modifier; /// For example, n in varchar(n)
for (uint16_t i = 0; i < num_columns; ++i)
{
String column_name;
@ -451,23 +489,14 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl
data_type_id = readInt32(replication_message, pos, size);
type_modifier = readInt32(replication_message, pos, size);
if (new_relation_definition)
if (buffer.attributes[i].atttypid != data_type_id || buffer.attributes[i].atttypmod != type_modifier)
{
current_schema_data.column_identifiers.emplace_back(std::make_pair(data_type_id, type_modifier));
}
else
{
if (current_schema_data.column_identifiers[i].first != data_type_id
|| current_schema_data.column_identifiers[i].second != type_modifier)
{
markTableAsSkipped(relation_id, table_name);
return;
}
markTableAsSkipped(relation_id, table_name);
return;
}
}
tables_to_sync.insert(table_name);
break;
}
case 'O': // Origin
@ -489,19 +518,19 @@ void MaterializedPostgreSQLConsumer::syncTables()
{
for (const auto & table_name : tables_to_sync)
{
auto & buffer = buffers.find(table_name)->second;
Block result_rows = buffer.description.sample_block.cloneWithColumns(std::move(buffer.columns));
auto & storage_data = storages.find(table_name)->second;
Block result_rows = storage_data.buffer.description.sample_block.cloneWithColumns(std::move(storage_data.buffer.columns));
if (result_rows.rows())
{
auto storage = storages[table_name];
auto storage = storage_data.storage;
auto insert_context = Context::createCopy(context);
insert_context->setInternalQuery(true);
auto insert = std::make_shared<ASTInsertQuery>();
insert->table_id = storage->getStorageID();
insert->columns = buffer.columnsAST;
insert->columns = storage_data.buffer.columns_ast;
InterpreterInsertQuery interpreter(insert, insert_context, true);
auto io = interpreter.execute();
@ -514,7 +543,7 @@ void MaterializedPostgreSQLConsumer::syncTables()
CompletedPipelineExecutor executor(io.pipeline);
executor.execute();
buffer.columns = buffer.description.sample_block.cloneEmptyColumns();
storage_data.buffer.columns = storage_data.buffer.description.sample_block.cloneEmptyColumns();
}
}
@ -599,34 +628,21 @@ bool MaterializedPostgreSQLConsumer::isSyncAllowed(Int32 relation_id, const Stri
void MaterializedPostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const String & relation_name)
{
/// Empty lsn string means - continue waiting for valid lsn.
skip_list.insert({relation_id, ""});
skip_list.insert({relation_id, ""}); /// Empty lsn string means - continue waiting for valid lsn.
storages.erase(relation_name);
if (storages.count(relation_name))
{
/// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream
/// and it receives first data after update.
schema_data.erase(relation_id);
/// Clear table buffer.
auto & buffer = buffers.find(relation_name)->second;
buffer.columns = buffer.description.sample_block.cloneEmptyColumns();
if (allow_automatic_update)
LOG_TRACE(log, "Table {} (relation_id: {}) is skipped temporarily. It will be reloaded in the background", relation_name, relation_id);
else
LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name, relation_id);
}
if (allow_automatic_update)
LOG_TRACE(log, "Table {} (relation_id: {}) is skipped temporarily. It will be reloaded in the background", relation_name, relation_id);
else
LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name, relation_id);
}
void MaterializedPostgreSQLConsumer::addNested(const String & postgres_table_name, StoragePtr nested_storage, const String & table_start_lsn)
void MaterializedPostgreSQLConsumer::addNested(
const String & postgres_table_name, StorageInfo nested_storage_info, const String & table_start_lsn)
{
/// Cache new pointer to replacingMergeTree table.
storages.emplace(postgres_table_name, nested_storage);
/// Add new in-memory buffer.
buffers.emplace(postgres_table_name, Buffer(nested_storage));
assert(!storages.contains(postgres_table_name));
storages.emplace(postgres_table_name, nested_storage_info);
/// Replication consumer will read wall and check for currently processed table whether it is allowed to start applying
/// changes to this table.
@ -634,14 +650,10 @@ void MaterializedPostgreSQLConsumer::addNested(const String & postgres_table_nam
}
void MaterializedPostgreSQLConsumer::updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn)
void MaterializedPostgreSQLConsumer::updateNested(const String & table_name, StorageInfo nested_storage_info, Int32 table_id, const String & table_start_lsn)
{
/// Cache new pointer to replacingMergeTree table.
storages[table_name] = nested_storage;
/// Create a new empty buffer (with updated metadata), where data is first loaded before syncing into actual table.
auto & buffer = buffers.find(table_name)->second;
buffer.createEmptyBuffer(nested_storage);
assert(!storages.contains(table_name));
storages.emplace(table_name, nested_storage_info);
/// Set start position to valid lsn. Before it was an empty string. Further read for table allowed, if it has a valid lsn.
skip_list[table_id] = table_start_lsn;
@ -651,7 +663,6 @@ void MaterializedPostgreSQLConsumer::updateNested(const String & table_name, Sto
void MaterializedPostgreSQLConsumer::removeNested(const String & postgres_table_name)
{
storages.erase(postgres_table_name);
buffers.erase(postgres_table_name);
deleted_tables.insert(postgres_table_name);
}
@ -706,7 +717,17 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot()
current_lsn = (*row)[0];
lsn_value = getLSNValue(current_lsn);
processReplicationMessage((*row)[1].c_str(), (*row)[1].size());
try
{
// LOG_DEBUG(log, "Current message: {}", (*row)[1]);
processReplicationMessage((*row)[1].c_str(), (*row)[1].size());
}
catch (const Exception & e)
{
if (e.code() == ErrorCodes::POSTGRESQL_REPLICATION_INTERNAL_ERROR)
continue;
throw;
}
}
}
catch (const Exception &)
@ -737,11 +758,6 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot()
LOG_ERROR(log, "Conversion error: {}", e.what());
return false;
}
catch (const pqxx::in_doubt_error & e)
{
LOG_ERROR(log, "PostgreSQL library has some doubts: {}", e.what());
return false;
}
catch (const pqxx::internal_error & e)
{
LOG_ERROR(log, "PostgreSQL library internal error: {}", e.what());
@ -749,16 +765,8 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot()
}
catch (...)
{
/// Since reading is done from a background task, it is important to catch any possible error
/// in order to understand why something does not work.
try
{
std::rethrow_exception(std::current_exception());
}
catch (const std::exception& e)
{
LOG_ERROR(log, "Unexpected error: {}", e.what());
}
tryLogCurrentException(__PRETTY_FUNCTION__);
return false;
}
if (!tables_to_sync.empty())
@ -770,6 +778,11 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot()
bool MaterializedPostgreSQLConsumer::consume(std::vector<std::pair<Int32, String>> & skipped_tables)
{
/// Read up to max_block_size changed (approximately - in same cases might be more).
/// false: no data was read, reschedule.
/// true: some data was read, schedule as soon as possible.
auto read_next = readFromReplicationSlot();
/// Check if there are tables, which are skipped from being updated by changes from replication stream,
/// because schema changes were detected. Update them, if it is allowed.
if (allow_automatic_update && !skip_list.empty())
@ -786,10 +799,6 @@ bool MaterializedPostgreSQLConsumer::consume(std::vector<std::pair<Int32, String
}
}
/// Read up to max_block_size changed (approximately - in same cases might be more).
/// false: no data was read, reschedule.
/// true: some data was read, schedule as soon as possible.
auto read_next = readFromReplicationSlot();
return read_next;
}

View File

@ -8,36 +8,78 @@
#include <base/logger_useful.h>
#include <Storages/IStorage.h>
#include <Parsers/ASTExpressionList.h>
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
namespace DB
{
struct SettingChange;
struct StorageInfo
{
StoragePtr storage;
PostgreSQLTableStructure::Attributes attributes;
StorageInfo(StoragePtr storage_, const PostgreSQLTableStructure::Attributes & attributes_)
: storage(storage_), attributes(attributes_) {}
};
using StorageInfos = std::unordered_map<String, StorageInfo>;
class MaterializedPostgreSQLConsumer
{
public:
using Storages = std::unordered_map<String, StoragePtr>;
private:
struct StorageData
{
struct Buffer
{
ExternalResultDescription description;
MutableColumns columns;
/// Needed to pass to insert query columns list in syncTables().
std::shared_ptr<ASTExpressionList> columns_ast;
/// Needed for insertPostgreSQLValue() method to parse array
std::unordered_map<size_t, PostgreSQLArrayInfo> array_info;
/// To validate ddl.
PostgreSQLTableStructure::Attributes attributes;
Buffer(StorageMetadataPtr storage_metadata, const PostgreSQLTableStructure::Attributes & attributes_);
size_t getColumnsNum() const
{
const auto & sample_block = description.sample_block;
return sample_block.columns();
}
};
StoragePtr storage;
Buffer buffer;
explicit StorageData(const StorageInfo & storage_info);
StorageData(const StorageData & other) = delete;
};
using Storages = std::unordered_map<String, StorageData>;
public:
MaterializedPostgreSQLConsumer(
ContextPtr context_,
std::shared_ptr<postgres::Connection> connection_,
const String & replication_slot_name_,
const String & publication_name_,
const String & start_lsn,
const size_t max_block_size_,
size_t max_block_size_,
bool schema_as_a_part_of_table_name_,
bool allow_automatic_update_,
Storages storages_,
StorageInfos storages_,
const String & name_for_logger);
bool consume(std::vector<std::pair<Int32, String>> & skipped_tables);
/// Called from reloadFromSnapshot by replication handler. This method is needed to move a table back into synchronization
/// process if it was skipped due to schema changes.
void updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn);
void updateNested(const String & table_name, StorageInfo nested_storage_info, Int32 table_id, const String & table_start_lsn);
void addNested(const String & postgres_table_name, StoragePtr nested_storage, const String & table_start_lsn);
void addNested(const String & postgres_table_name, StorageInfo nested_storage_info, const String & table_start_lsn);
void removeNested(const String & postgres_table_name);
@ -55,25 +97,8 @@ private:
bool isSyncAllowed(Int32 relation_id, const String & relation_name);
struct Buffer
{
ExternalResultDescription description;
MutableColumns columns;
/// Needed to pass to insert query columns list in syncTables().
std::shared_ptr<ASTExpressionList> columnsAST;
/// Needed for insertPostgreSQLValue() method to parse array
std::unordered_map<size_t, PostgreSQLArrayInfo> array_info;
Buffer(StoragePtr storage) { createEmptyBuffer(storage); }
void createEmptyBuffer(StoragePtr storage);
};
using Buffers = std::unordered_map<String, Buffer>;
static void insertDefaultValue(Buffer & buffer, size_t column_idx);
static void insertValue(Buffer & buffer, const std::string & value, size_t column_idx);
static void insertDefaultValue(StorageData::Buffer & buffer, size_t column_idx);
void insertValue(StorageData::Buffer & buffer, const std::string & value, size_t column_idx);
enum class PostgreSQLQuery
{
@ -82,7 +107,7 @@ private:
DELETE
};
void readTupleData(Buffer & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false);
void readTupleData(StorageData::Buffer & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false);
template<typename T>
static T unhexN(const char * message, size_t pos, size_t n);
@ -95,7 +120,7 @@ private:
void markTableAsSkipped(Int32 relation_id, const String & relation_name);
/// lsn - log sequnce nuumber, like wal offset (64 bit).
Int64 getLSNValue(const std::string & lsn)
static Int64 getLSNValue(const std::string & lsn)
{
UInt32 upper_half, lower_half;
std::sscanf(lsn.data(), "%X/%X", &upper_half, &lower_half);
@ -125,28 +150,11 @@ private:
/// Holds `postgres_table_name` set.
std::unordered_set<std::string> tables_to_sync;
/// `postgres_table_name` -> ReplacingMergeTree table.
/// `postgres_table_name` -> StorageData.
Storages storages;
/// `postgres_table_name` -> In-memory buffer.
Buffers buffers;
std::unordered_map<Int32, String> relation_id_to_name;
struct SchemaData
{
Int16 number_of_columns;
/// data_type_id and type_modifier
std::vector<std::pair<Int32, Int32>> column_identifiers;
SchemaData(Int16 number_of_columns_) : number_of_columns(number_of_columns_) {}
};
/// Cache for table schema data to be able to detect schema changes, because ddl is not
/// replicated with postgresql logical replication protocol, but some table schema info
/// is received if it is the first time we received dml message for given relation in current session or
/// if relation definition has changed since the last relation definition message.
std::unordered_map<Int32, SchemaData> schema_data;
/// `postgres_relation_id` -> `start_lsn`
/// skip_list contains relation ids for tables on which ddl was performed, which can break synchronization.
/// This breaking changes are detected in replication stream in according replication message and table is added to skip list.

View File

@ -191,7 +191,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error)
createPublicationIfNeeded(tx);
/// List of nested tables (table_name -> nested_storage), which is passed to replication consumer.
std::unordered_map<String, StoragePtr> nested_storages;
std::unordered_map<String, StorageInfo> nested_storages;
/// snapshot_name is initialized only if a new replication slot is created.
/// start_lsn is initialized in two places:
@ -226,7 +226,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error)
{
try
{
nested_storages[table_name] = loadFromSnapshot(*tmp_connection, snapshot_name, table_name, storage->as<StorageMaterializedPostgreSQL>());
nested_storages.emplace(table_name, loadFromSnapshot(*tmp_connection, snapshot_name, table_name, storage->as<StorageMaterializedPostgreSQL>()));
}
catch (Exception & e)
{
@ -268,7 +268,12 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error)
auto * materialized_storage = storage->as <StorageMaterializedPostgreSQL>();
try
{
nested_storages[table_name] = materialized_storage->getNested();
auto [postgres_table_schema, postgres_table_name] = getSchemaAndTableName(table_name);
auto table_structure = fetchPostgreSQLTableStructure(tx, postgres_table_name, postgres_table_schema, true, true, true);
if (!table_structure.physical_columns)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns");
auto storage_info = StorageInfo(materialized_storage->getNested(), table_structure.physical_columns->attributes);
nested_storages.emplace(table_name, std::move(storage_info));
}
catch (Exception & e)
{
@ -321,7 +326,7 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ
}
StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & connection, String & snapshot_name, const String & table_name,
StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & connection, String & snapshot_name, const String & table_name,
StorageMaterializedPostgreSQL * materialized_storage)
{
auto tx = std::make_shared<pqxx::ReplicationTransaction>(connection.getRef());
@ -335,8 +340,13 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection &
query_str = fmt::format("SELECT * FROM {}", quoted_name);
LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name);
auto table_structure = fetchTableStructure(*tx, table_name);
if (!table_structure->physical_columns)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No table attributes");
auto table_attributes = table_structure->physical_columns->attributes;
auto table_override = tryGetTableOverride(current_database_name, table_name);
materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name), table_override ? table_override->as<ASTTableOverride>() : nullptr);
materialized_storage->createNestedIfNeeded(std::move(table_structure), table_override ? table_override->as<ASTTableOverride>() : nullptr);
auto nested_storage = materialized_storage->getNested();
auto insert = std::make_shared<ASTInsertQuery>();
@ -361,7 +371,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection &
auto nested_table_id = nested_storage->getStorageID();
LOG_DEBUG(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid));
return nested_storage;
return StorageInfo(nested_storage, std::move(table_attributes));
}
@ -793,9 +803,6 @@ std::set<String> PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::
PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure(
pqxx::ReplicationTransaction & tx, const std::string & table_name) const
{
if (!is_materialized_postgresql_database)
return nullptr;
PostgreSQLTableStructure structure;
try
{
@ -821,7 +828,7 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost
LOG_TRACE(log, "Adding table `{}` to replication", postgres_table_name);
postgres::Connection replication_connection(connection_info, /* replication */true);
String snapshot_name, start_lsn;
StoragePtr nested_storage;
StorageInfo nested_storage_info{ nullptr, {} };
{
auto tx = std::make_shared<pqxx::nontransaction>(replication_connection.getRef());
@ -837,8 +844,8 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost
throw Exception(ErrorCodes::LOGICAL_ERROR, "Internal table was not created");
postgres::Connection tmp_connection(connection_info);
nested_storage = loadFromSnapshot(tmp_connection, snapshot_name, postgres_table_name, materialized_storage);
materialized_storage->set(nested_storage);
nested_storage_info = loadFromSnapshot(tmp_connection, snapshot_name, postgres_table_name, materialized_storage);
materialized_storage->set(nested_storage_info.storage);
}
{
@ -847,7 +854,7 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost
}
/// Pass storage to consumer and lsn position, from which to start receiving replication messages for this table.
consumer->addNested(postgres_table_name, nested_storage, start_lsn);
consumer->addNested(postgres_table_name, nested_storage_info, start_lsn);
LOG_TRACE(log, "Table `{}` successfully added to replication", postgres_table_name);
}
catch (...)
@ -920,8 +927,8 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector<std::pai
auto temp_materialized_storage = materialized_storage->createTemporary();
/// This snapshot is valid up to the end of the transaction, which exported it.
StoragePtr temp_nested_storage = loadFromSnapshot(tmp_connection, snapshot_name, table_name,
temp_materialized_storage->as <StorageMaterializedPostgreSQL>());
auto [temp_nested_storage, table_attributes] = loadFromSnapshot(
tmp_connection, snapshot_name, table_name, temp_materialized_storage->as <StorageMaterializedPostgreSQL>());
auto table_id = materialized_storage->getNestedStorageID();
auto temp_table_id = temp_nested_storage->getStorageID();
@ -955,7 +962,7 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector<std::pai
nested_storage->getStorageID().getNameForLogs(), nested_sample_block.dumpStructure());
/// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position.
consumer->updateNested(table_name, nested_storage, relation_id, start_lsn);
consumer->updateNested(table_name, StorageInfo(nested_storage, std::move(table_attributes)), relation_id, start_lsn);
auto table_to_drop = DatabaseCatalog::instance().getTable(StorageID(temp_table_id.database_name, temp_table_id.table_name, table_id.uuid), nested_context);
auto drop_table_id = table_to_drop->getStorageID();

View File

@ -87,7 +87,7 @@ private:
void consumerFunc();
StoragePtr loadFromSnapshot(postgres::Connection & connection, std::string & snapshot_name, const String & table_name, StorageMaterializedPostgreSQL * materialized_storage);
StorageInfo loadFromSnapshot(postgres::Connection & connection, std::string & snapshot_name, const String & table_name, StorageMaterializedPostgreSQL * materialized_storage);
void reloadFromSnapshot(const std::vector<std::pair<Int32, String>> & relation_data);

View File

@ -346,7 +346,7 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d
ast_expression->name = "DateTime64";
ast_expression->arguments = std::make_shared<ASTExpressionList>();
ast_expression->arguments->children.emplace_back(std::make_shared<ASTLiteral>(UInt32(6)));
return ast_expression;
return std::move(ast_expression);
}
return std::make_shared<ASTIdentifier>(data_type->getName());
@ -404,7 +404,7 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(
table_id.database_name, table_id.table_name);
}
if (!table_structure->columns && (!table_override || !table_override->columns))
if (!table_structure->physical_columns && (!table_override || !table_override->columns))
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns returned for table {}.{}",
table_id.database_name, table_id.table_name);
@ -446,7 +446,7 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(
}
else
{
ordinary_columns_and_types = *table_structure->columns;
ordinary_columns_and_types = table_structure->physical_columns->columns;
columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types));
}
@ -456,7 +456,7 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(
}
else
{
ordinary_columns_and_types = *table_structure->columns;
ordinary_columns_and_types = table_structure->physical_columns->columns;
columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types));
}
@ -466,9 +466,9 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(
NamesAndTypesList merging_columns;
if (table_structure->primary_key_columns)
merging_columns = *table_structure->primary_key_columns;
merging_columns = table_structure->primary_key_columns->columns;
else
merging_columns = *table_structure->replica_identity_columns;
merging_columns = table_structure->replica_identity_columns->columns;
order_by_expression->name = "tuple";
order_by_expression->arguments = std::make_shared<ASTExpressionList>();
@ -505,7 +505,7 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(
storage_metadata.setConstraints(constraints);
setInMemoryMetadata(storage_metadata);
return create_table_query;
return std::move(create_table_query);
}

View File

@ -442,6 +442,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
createNewZooKeeperNodes();
syncPinnedPartUUIDs();
createTableSharedID();
}
@ -7025,12 +7027,53 @@ void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded()
background_moves_assignee.start();
}
std::unique_ptr<MergeTreeSettings> StorageReplicatedMergeTree::getDefaultSettings() const
{
return std::make_unique<MergeTreeSettings>(getContext()->getReplicatedMergeTreeSettings());
}
String StorageReplicatedMergeTree::getTableSharedID() const
{
return toString(table_shared_id);
}
void StorageReplicatedMergeTree::createTableSharedID()
{
if (table_shared_id != UUIDHelpers::Nil)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table shared id already initialized");
zkutil::ZooKeeperPtr zookeeper = getZooKeeper();
String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_shared_id";
String id;
if (!zookeeper->tryGet(zookeeper_table_id_path, id))
{
UUID table_id_candidate;
auto storage_id = getStorageID();
if (storage_id.uuid != UUIDHelpers::Nil)
table_id_candidate = storage_id.uuid;
else
table_id_candidate = UUIDHelpers::generateV4();
id = toString(table_id_candidate);
auto code = zookeeper->tryCreate(zookeeper_table_id_path, id, zkutil::CreateMode::Persistent);
if (code == Coordination::Error::ZNODEEXISTS)
{ /// Other replica create node early
id = zookeeper->get(zookeeper_table_id_path);
}
else if (code != Coordination::Error::ZOK)
{
throw zkutil::KeeperException(code, zookeeper_table_id_path);
}
}
table_shared_id = parseFromString<UUID>(id);
}
void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part) const
{
if (!part.volume)
@ -7038,7 +7081,6 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part)
DiskPtr disk = part.volume->getDisk();
if (!disk || !disk->supportZeroCopyReplication())
return;
String zero_copy = fmt::format("zero_copy_{}", toString(disk->getType()));
zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper();
if (!zookeeper)
@ -7047,73 +7089,100 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part)
String id = part.getUniqueId();
boost::replace_all(id, "/", "_");
String zookeeper_node = fs::path(zookeeper_path) / zero_copy / "shared" / part.name / id / replica_name;
LOG_TRACE(log, "Set zookeeper lock {}", zookeeper_node);
/// In rare case other replica can remove path between createAncestors and createIfNotExists
/// So we make up to 5 attempts
for (int attempts = 5; attempts > 0; --attempts)
Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), disk->getType(), getTableSharedID(),
part.name, zookeeper_path);
for (const auto & zc_zookeeper_path : zc_zookeeper_paths)
{
try
{
zookeeper->createAncestors(zookeeper_node);
zookeeper->createIfNotExists(zookeeper_node, "lock");
break;
}
catch (const zkutil::KeeperException & e)
{
if (e.code == Coordination::Error::ZNONODE)
continue;
throw;
}
String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name;
LOG_TRACE(log, "Set zookeeper lock {}", zookeeper_node);
createZeroCopyLockNode(zookeeper, zookeeper_node);
}
}
bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const
{
return unlockSharedData(part, part.name);
}
bool StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, const String & name) const
{
if (!part.volume)
return true;
DiskPtr disk = part.volume->getDisk();
if (!disk || !disk->supportZeroCopyReplication())
return true;
String zero_copy = fmt::format("zero_copy_{}", toString(disk->getType()));
zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper();
if (!zookeeper)
return true;
String id = part.getUniqueId();
auto ref_count = part.getNumberOfRefereneces();
if (ref_count > 0) /// Keep part shard info for frozen backups
return false;
return unlockSharedDataByID(part.getUniqueId(), getTableSharedID(), name, replica_name, disk, zookeeper, *getSettings(), log,
zookeeper_path);
}
bool StorageReplicatedMergeTree::unlockSharedDataByID(String id, const String & table_uuid, const String & part_name,
const String & replica_name_, DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings,
Poco::Logger * logger, const String & zookeeper_path_old)
{
boost::replace_all(id, "/", "_");
String zookeeper_part_node = fs::path(zookeeper_path) / zero_copy / "shared" / part.name;
String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id;
String zookeeper_node = fs::path(zookeeper_part_uniq_node) / replica_name;
Strings zc_zookeeper_paths = getZeroCopyPartPath(settings, disk->getType(), table_uuid, part_name, zookeeper_path_old);
LOG_TRACE(log, "Remove zookeeper lock {}", zookeeper_node);
bool res = true;
zookeeper->tryRemove(zookeeper_node);
Strings children;
zookeeper->tryGetChildren(zookeeper_part_uniq_node, children);
if (!children.empty())
for (const auto & zc_zookeeper_path : zc_zookeeper_paths)
{
LOG_TRACE(log, "Found zookeper locks for {}", zookeeper_part_uniq_node);
return false;
String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / id;
String zookeeper_node = fs::path(zookeeper_part_uniq_node) / replica_name_;
LOG_TRACE(logger, "Remove zookeeper lock {}", zookeeper_node);
zookeeper_ptr->tryRemove(zookeeper_node);
Strings children;
zookeeper_ptr->tryGetChildren(zookeeper_part_uniq_node, children);
if (!children.empty())
{
LOG_TRACE(logger, "Found zookeper locks for {}", zookeeper_part_uniq_node);
res = false;
continue;
}
auto e = zookeeper_ptr->tryRemove(zookeeper_part_uniq_node);
LOG_TRACE(logger, "Remove parent zookeeper lock {} : {}", zookeeper_part_uniq_node, e != Coordination::Error::ZNOTEMPTY);
/// Even when we have lock with same part name, but with different uniq, we can remove files on S3
children.clear();
String zookeeper_part_node = fs::path(zookeeper_part_uniq_node).parent_path();
zookeeper_ptr->tryGetChildren(zookeeper_part_node, children);
if (children.empty())
{
/// Cleanup after last uniq removing
e = zookeeper_ptr->tryRemove(zookeeper_part_node);
LOG_TRACE(logger, "Remove parent zookeeper lock {} : {}", zookeeper_part_node, e != Coordination::Error::ZNOTEMPTY);
}
else
{
LOG_TRACE(logger, "Can't remove parent zookeeper lock {} : {}", zookeeper_part_node, children.size());
for (auto & c : children)
{
LOG_TRACE(logger, "Child node {}", c);
}
}
}
zookeeper->tryRemove(zookeeper_part_uniq_node);
/// Even when we have lock with same part name, but with different uniq, we can remove files on S3
children.clear();
zookeeper->tryGetChildren(zookeeper_part_node, children);
if (children.empty())
/// Cleanup after last uniq removing
zookeeper->tryRemove(zookeeper_part_node);
return true;
return res;
}
@ -7146,20 +7215,24 @@ String StorageReplicatedMergeTree::getSharedDataReplica(
if (!zookeeper)
return best_replica;
String zero_copy = fmt::format("zero_copy_{}", toString(disk_type));
String zookeeper_part_node = fs::path(zookeeper_path) / zero_copy / "shared" / part.name;
Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), disk_type, getTableSharedID(), part.name,
zookeeper_path);
Strings ids;
zookeeper->tryGetChildren(zookeeper_part_node, ids);
std::set<String> replicas;
Strings replicas;
for (const auto & id : ids)
for (const auto & zc_zookeeper_path : zc_zookeeper_paths)
{
String zookeeper_part_uniq_node = fs::path(zookeeper_part_node) / id;
Strings id_replicas;
zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas);
LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size());
replicas.insert(replicas.end(), id_replicas.begin(), id_replicas.end());
Strings ids;
zookeeper->tryGetChildren(zc_zookeeper_path, ids);
for (const auto & id : ids)
{
String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / id;
Strings id_replicas;
zookeeper->tryGetChildren(zookeeper_part_uniq_node, id_replicas);
LOG_TRACE(log, "Found zookeper replicas for {}: {}", zookeeper_part_uniq_node, id_replicas.size());
replicas.insert(id_replicas.begin(), id_replicas.end());
}
}
LOG_TRACE(log, "Found zookeper replicas for part {}: {}", part.name, replicas.size());
@ -7212,24 +7285,45 @@ String StorageReplicatedMergeTree::getSharedDataReplica(
return best_replica;
}
String StorageReplicatedMergeTree::findReplicaHavingPart(
const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_)
Strings StorageReplicatedMergeTree::getZeroCopyPartPath(const MergeTreeSettings & settings, DiskType disk_type, const String & table_uuid,
const String & part_name, const String & zookeeper_path_old)
{
Strings replicas = zookeeper_->getChildren(fs::path(zookeeper_path_) / "replicas");
Strings res;
String zero_copy = fmt::format("zero_copy_{}", toString(disk_type));
String new_path = fs::path(settings.remote_fs_zero_copy_zookeeper_path.toString()) / zero_copy / table_uuid / part_name;
res.push_back(new_path);
if (settings.remote_fs_zero_copy_path_compatible_mode && !zookeeper_path_old.empty())
{ /// Compatibility mode for cluster with old and new versions
String old_path = fs::path(zookeeper_path_old) / zero_copy / "shared" / part_name;
res.push_back(old_path);
}
return res;
}
String StorageReplicatedMergeTree::findReplicaHavingPart(
const String & part_name, const String & zookeeper_path_, zkutil::ZooKeeper::Ptr zookeeper_ptr)
{
Strings replicas = zookeeper_ptr->getChildren(fs::path(zookeeper_path_) / "replicas");
/// Select replicas in uniformly random order.
std::shuffle(replicas.begin(), replicas.end(), thread_local_rng);
for (const String & replica : replicas)
{
if (zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "parts" / part_name)
&& zookeeper_->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active"))
if (zookeeper_ptr->exists(fs::path(zookeeper_path_) / "replicas" / replica / "parts" / part_name)
&& zookeeper_ptr->exists(fs::path(zookeeper_path_) / "replicas" / replica / "is_active"))
return fs::path(zookeeper_path_) / "replicas" / replica;
}
return {};
}
bool StorageReplicatedMergeTree::checkIfDetachedPartExists(const String & part_name)
{
fs::directory_iterator dir_end;
@ -7240,6 +7334,7 @@ bool StorageReplicatedMergeTree::checkIfDetachedPartExists(const String & part_n
return false;
}
bool StorageReplicatedMergeTree::checkIfDetachedPartitionExists(const String & partition_name)
{
fs::directory_iterator dir_end;
@ -7430,4 +7525,180 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP
return true;
}
void StorageReplicatedMergeTree::createZeroCopyLockNode(const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node)
{
/// In rare case other replica can remove path between createAncestors and createIfNotExists
/// So we make up to 5 attempts
for (int attempts = 5; attempts > 0; --attempts)
{
try
{
zookeeper->createAncestors(zookeeper_node);
zookeeper->createIfNotExists(zookeeper_node, "lock");
break;
}
catch (const zkutil::KeeperException & e)
{
if (e.code == Coordination::Error::ZNONODE)
continue;
throw;
}
}
}
namespace
{
/// Special metadata used during freeze table. Required for zero-copy
/// replication.
struct FreezeMetaData
{
public:
void fill(const StorageReplicatedMergeTree & storage)
{
is_replicated = storage.supportsReplication();
is_remote = storage.isRemote();
replica_name = storage.getReplicaName();
zookeeper_name = storage.getZooKeeperName();
table_shared_id = storage.getTableSharedID();
}
void save(DiskPtr disk, const String & path) const
{
auto file_path = getFileName(path);
auto buffer = disk->writeMetaFile(file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
writeIntText(version, *buffer);
buffer->write("\n", 1);
writeBoolText(is_replicated, *buffer);
buffer->write("\n", 1);
writeBoolText(is_remote, *buffer);
buffer->write("\n", 1);
writeString(replica_name, *buffer);
buffer->write("\n", 1);
writeString(zookeeper_name, *buffer);
buffer->write("\n", 1);
writeString(table_shared_id, *buffer);
buffer->write("\n", 1);
}
bool load(DiskPtr disk, const String & path)
{
auto file_path = getFileName(path);
if (!disk->exists(file_path))
return false;
auto buffer = disk->readMetaFile(file_path, ReadSettings(), {});
readIntText(version, *buffer);
if (version != 1)
{
LOG_ERROR(&Poco::Logger::get("FreezeMetaData"), "Unknown freezed metadata version: {}", version);
return false;
}
DB::assertChar('\n', *buffer);
readBoolText(is_replicated, *buffer);
DB::assertChar('\n', *buffer);
readBoolText(is_remote, *buffer);
DB::assertChar('\n', *buffer);
readString(replica_name, *buffer);
DB::assertChar('\n', *buffer);
readString(zookeeper_name, *buffer);
DB::assertChar('\n', *buffer);
readString(table_shared_id, *buffer);
DB::assertChar('\n', *buffer);
return true;
}
static void clean(DiskPtr disk, const String & path)
{
disk->removeMetaFileIfExists(getFileName(path));
}
private:
static String getFileName(const String & path)
{
return fs::path(path) / "frozen_metadata.txt";
}
public:
int version = 1;
bool is_replicated;
bool is_remote;
String replica_name;
String zookeeper_name;
String table_shared_id;
};
}
bool StorageReplicatedMergeTree::removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed)
{
if (disk->supportZeroCopyReplication())
{
if (is_freezed)
{
FreezeMetaData meta;
if (meta.load(disk, path))
{
FreezeMetaData::clean(disk, path);
return removeSharedDetachedPart(disk, path, part_name, meta.table_shared_id, meta.zookeeper_name, meta.replica_name, "");
}
}
else
{
String table_id = getTableSharedID();
return removeSharedDetachedPart(disk, path, part_name, table_id, zookeeper_name, replica_name, zookeeper_path);
}
}
disk->removeRecursive(path);
return false;
}
bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
const String &, const String & detached_replica_name, const String & detached_zookeeper_path)
{
bool keep_shared = false;
zkutil::ZooKeeperPtr zookeeper = getZooKeeper();
if (zookeeper)
{
fs::path checksums = fs::path(path) / "checksums.txt";
if (disk->exists(checksums))
{
auto ref_count = disk->getRefCount(checksums);
if (ref_count == 0)
{
String id = disk->getUniqueId(checksums);
keep_shared = !StorageReplicatedMergeTree::unlockSharedDataByID(id, table_uuid, part_name,
detached_replica_name, disk, zookeeper, getContext()->getReplicatedMergeTreeSettings(), log,
detached_zookeeper_path);
}
else
keep_shared = true;
}
}
disk->removeSharedRecursive(path, keep_shared);
return keep_shared;
}
void StorageReplicatedMergeTree::createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr, String backup_part_path) const
{
if (disk->supportZeroCopyReplication())
{
FreezeMetaData meta;
meta.fill(*this);
meta.save(disk, backup_part_path);
}
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <base/shared_ptr_helper.h>
#include <base/UUID.h>
#include <atomic>
#include <pcg_random.hpp>
#include <Storages/IStorage.h>
@ -236,6 +237,16 @@ public:
/// Return false if data is still used by another node
bool unlockSharedData(const IMergeTreeDataPart & part) const override;
/// Remove lock with old name for shared data part after rename
bool unlockSharedData(const IMergeTreeDataPart & part, const String & name) const override;
/// Unlock shared data part in zookeeper by part id
/// Return true if data unlocked
/// Return false if data is still used by another node
static bool unlockSharedDataByID(String id, const String & table_uuid, const String & part_name, const String & replica_name_,
DiskPtr disk, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger,
const String & zookeeper_path_old);
/// Fetch part only if some replica has it on shared storage like S3
bool tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override;
@ -263,6 +274,12 @@ public:
bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name);
// Return default or custom zookeeper name for table
String getZooKeeperName() const { return zookeeper_name; }
// Return table id, common for different replicas
String getTableSharedID() const;
static const String getDefaultZooKeeperName() { return default_zookeeper_name; }
private:
@ -393,6 +410,9 @@ private:
ThrottlerPtr replicated_fetches_throttler;
ThrottlerPtr replicated_sends_throttler;
/// Global ID, synced via ZooKeeper between replicas
UUID table_shared_id;
template <class Func>
void foreachActiveParts(Func && func, bool select_sequential_consistency) const;
@ -722,6 +742,22 @@ private:
PartitionBlockNumbersHolder allocateBlockNumbersInAffectedPartitions(
const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const;
static Strings getZeroCopyPartPath(const MergeTreeSettings & settings, DiskType disk_type, const String & table_uuid,
const String & part_name, const String & zookeeper_path_old);
static void createZeroCopyLockNode(const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node);
bool removeDetachedPart(DiskPtr disk, const String & path, const String & part_name, bool is_freezed) override;
bool removeSharedDetachedPart(DiskPtr disk, const String & path, const String & part_name, const String & table_uuid,
const String & zookeeper_name, const String & replica_name, const String & zookeeper_path);
/// Create freeze metadata for table and save in zookeeper. Required only if zero-copy replication enabled.
void createAndStoreFreezeMetadata(DiskPtr disk, DataPartPtr part, String backup_part_path) const override;
// Create table id if needed
void createTableSharedID();
protected:
/** If not 'attach', either creates a new table in ZK, or adds a replica to an existing table.
*/

View File

@ -45,13 +45,13 @@ ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr c
{
const bool use_nulls = context->getSettingsRef().external_table_functions_use_nulls;
auto connection_holder = connection_pool->get();
auto columns = fetchPostgreSQLTableStructure(
connection_holder->get(), configuration->table, configuration->schema, use_nulls).columns;
auto columns_info = fetchPostgreSQLTableStructure(
connection_holder->get(), configuration->table, configuration->schema, use_nulls).physical_columns;
if (!columns)
if (!columns_info)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table structure not returned");
return ColumnsDescription{*columns};
return ColumnsDescription{columns_info->columns};
}

View File

@ -69,6 +69,7 @@ TRUSTED_CONTRIBUTORS = {e.lower() for e in [
"zlobober", # Developer of YT
"ilejn", # Arenadata, responsible for Kerberized Kafka
"thomoco", # ClickHouse
"BoloniniD", # Seasoned contributor, HSE
]}

View File

@ -93,6 +93,7 @@ TRUSTED_CONTRIBUTORS = {e.lower() for e in [
"vzakaznikov",
"YiuRULE",
"zlobober", # Developer of YT
"BoloniniD", # Seasoned contributor, HSE
]}

View File

@ -2256,7 +2256,7 @@ class ClickHouseInstance:
logging.debug('{} log line(s) matching "{}" appeared in a {:.3f} seconds'.format(repetitions, regexp, wait_duration))
return wait_duration
def file_exists(self, path):
def path_exists(self, path):
return self.exec_in_container(
["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n'
@ -2694,6 +2694,20 @@ class ClickHouseInstance:
if p.exists(self.path):
shutil.rmtree(self.path)
def wait_for_path_exists(self, path, seconds):
while seconds > 0:
seconds -= 1
if self.path_exists(path):
return
time.sleep(1)
def get_backuped_s3_objects(self, disk, backup_name):
path = f'/var/lib/clickhouse/disks/{disk}/shadow/{backup_name}/store'
self.wait_for_path_exists(path, 10)
command = ['find', path, '-type', 'f',
'-exec', 'grep', '-o', 'r[01]\\{64\\}-file-[[:lower:]]\\{32\\}', '{}', ';']
return self.exec_in_container(command).split('\n')
class ClickHouseKiller(object):
def __init__(self, clickhouse_node):

View File

@ -203,7 +203,7 @@ def test_reload_after_fail_by_timer(started_cluster):
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/file.txt"),
"/etc/clickhouse-server/dictionaries/no_file_2.txt")
# Check that file appears in container and wait if needed.
while not instance.file_exists("/etc/clickhouse-server/dictionaries/no_file_2.txt"):
while not instance.path_exists("/etc/clickhouse-server/dictionaries/no_file_2.txt"):
time.sleep(1)
assert("9\t10\n" == instance.exec_in_container(["cat", "/etc/clickhouse-server/dictionaries/no_file_2.txt"]))
instance.query("SYSTEM RELOAD DICTIONARY no_file_2")

View File

@ -36,6 +36,38 @@ def get_genuine_zk():
get_fake_zk
]
)
def test_remove_acl(started_cluster, get_zk):
auth_connection = get_zk()
auth_connection.add_auth('digest', 'user1:password1')
# Consistent with zookeeper, accept generated digest
auth_connection.create("/test_remove_acl1", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", read=True, write=False, create=False, delete=False, admin=False)])
auth_connection.create("/test_remove_acl2", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", read=True, write=True, create=False, delete=False, admin=False)])
auth_connection.create("/test_remove_acl3", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)])
auth_connection.delete("/test_remove_acl2")
auth_connection.create("/test_remove_acl4", b"dataX", acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", read=True, write=True, create=True, delete=False, admin=False)])
acls, stat = auth_connection.get_acls("/test_remove_acl3")
assert stat.aversion == 0
assert len(acls) == 1
for acl in acls:
assert acl.acl_list == ['ALL']
assert acl.perms == 31
@pytest.mark.parametrize(
('get_zk'),
[
get_genuine_zk,
get_fake_zk
]
)
def test_digest_auth_basic(started_cluster, get_zk):
auth_connection = get_zk()

View File

@ -18,7 +18,7 @@ def started_cluster():
def check_log_file():
assert node.file_exists("/var/log/clickhouse-server/clickhouse-server.log.lz4")
assert node.path_exists("/var/log/clickhouse-server/clickhouse-server.log.lz4")
lz4_output = node.exec_in_container(["bash", "-c", "lz4 -t /var/log/clickhouse-server/clickhouse-server.log.lz4 2>&1"], user='root')
assert lz4_output.count('Error') == 0, lz4_output

View File

@ -416,6 +416,53 @@ def test_table_override(started_cluster):
assert_eq_with_retry(instance, query, expected)
def test_table_schema_changes_2(started_cluster):
drop_materialized_db()
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
database=True)
cursor = conn.cursor()
table_name = "test_table"
create_postgres_table(cursor, table_name, template=postgres_table_template_2);
instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, number, number, number from numbers(25)")
create_materialized_db(ip=started_cluster.postgres_ip,
port=started_cluster.postgres_port,
settings=["materialized_postgresql_allow_automatic_update = 1, materialized_postgresql_tables_list='test_table'"])
instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, number, number, number from numbers(25, 25)")
check_tables_are_synchronized(table_name);
cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value1")
cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value2")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value1 Text")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value2 Text")
cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value3")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value3 Text")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value4 Text")
cursor.execute(f"UPDATE {table_name} SET value3 = 'kek' WHERE key%2=0")
check_tables_are_synchronized(table_name);
instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, toString(number), toString(number), toString(number), toString(number) from numbers(50, 25)")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value5 Integer")
cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value2")
instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, toString(number), toString(number), toString(number), number from numbers(75, 25)")
check_tables_are_synchronized(table_name);
instance.restart_clickhouse()
check_tables_are_synchronized(table_name);
cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value5")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value5 Text")
instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, toString(number), toString(number), toString(number), toString(number) from numbers(100, 25)")
check_tables_are_synchronized(table_name);
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value6 Text")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value7 Integer")
cursor.execute(f"ALTER TABLE {table_name} ADD COLUMN value8 Integer")
cursor.execute(f"ALTER TABLE {table_name} DROP COLUMN value5")
instance.query(f"INSERT INTO postgres_database.{table_name} SELECT number, toString(number), toString(number), toString(number), toString(number), number, number from numbers(125, 25)")
check_tables_are_synchronized(table_name);
if __name__ == '__main__':
cluster.start()
input("Cluster created, press any key to destroy...")

View File

@ -7,18 +7,21 @@
<endpoint>http://minio1:9001/root/data/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
<send_metadata>true</send_metadata>
</s31>
<s31_again>
<type>s3</type>
<endpoint>http://minio1:9001/root/data/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
<send_metadata>true</send_metadata>
</s31_again>
<s32>
<type>s3</type>
<endpoint>http://minio1:9001/root/data2/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
<send_metadata>true</send_metadata>
</s32>
</disks>
<policies>

View File

@ -32,11 +32,30 @@ def get_large_objects_count(cluster, size=100, folder='data'):
minio = cluster.minio_client
counter = 0
for obj in minio.list_objects(cluster.minio_bucket, '{}/'.format(folder)):
if obj.size >= size:
if obj.size is not None and obj.size >= size:
counter = counter + 1
return counter
def check_objects_exisis(cluster, object_list, folder='data'):
minio = cluster.minio_client
for obj in object_list:
if obj:
minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj))
def check_objects_not_exisis(cluster, object_list, folder='data'):
minio = cluster.minio_client
for obj in object_list:
if obj:
try:
minio.stat_object(cluster.minio_bucket, '{}/{}'.format(folder, obj))
except Exception as error:
assert "NoSuchKey" in str(error)
else:
assert False, "Object {} should not be exists".format(obj)
def wait_for_large_objects_count(cluster, expected, size=100, timeout=30):
while timeout > 0:
if get_large_objects_count(cluster, size=size) == expected:
@ -266,6 +285,138 @@ def test_s3_zero_copy_with_ttl_delete(cluster, large_data, iterations):
node2.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY")
def wait_mutations(node, table, seconds):
time.sleep(1)
while seconds > 0:
seconds -= 1
mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0")
if mutations == '0\n':
return
time.sleep(1)
mutations = node.query(f"SELECT count() FROM system.mutations WHERE table='{table}' AND is_done=0")
assert mutations == '0\n'
def test_s3_zero_copy_unfreeze(cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
node1.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY")
node2.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY")
node1.query(
"""
CREATE TABLE unfreeze_test ON CLUSTER test_cluster (d UInt64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/unfreeze_test', '{}')
ORDER BY d
SETTINGS storage_policy='s3'
"""
.format('{replica}')
)
node1.query("INSERT INTO unfreeze_test VALUES (0)")
node1.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'freeze_backup1'")
node2.query("ALTER TABLE unfreeze_test FREEZE WITH NAME 'freeze_backup2'")
wait_mutations(node1, "unfreeze_test", 10)
wait_mutations(node2, "unfreeze_test", 10)
objects01 = node1.get_backuped_s3_objects("s31", "freeze_backup1")
objects02 = node2.get_backuped_s3_objects("s31", "freeze_backup2")
assert objects01 == objects02
check_objects_exisis(cluster, objects01)
node1.query("TRUNCATE TABLE unfreeze_test")
objects11 = node1.get_backuped_s3_objects("s31", "freeze_backup1")
objects12 = node2.get_backuped_s3_objects("s31", "freeze_backup2")
assert objects01 == objects11
assert objects01 == objects12
check_objects_exisis(cluster, objects11)
node1.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'freeze_backup1'")
wait_mutations(node1, "unfreeze_test", 10)
check_objects_exisis(cluster, objects12)
node2.query("ALTER TABLE unfreeze_test UNFREEZE WITH NAME 'freeze_backup2'")
wait_mutations(node2, "unfreeze_test", 10)
check_objects_not_exisis(cluster, objects12)
node1.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY")
node2.query("DROP TABLE IF EXISTS unfreeze_test NO DELAY")
def test_s3_zero_copy_drop_detached(cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]
node1.query("DROP TABLE IF EXISTS drop_detached_test NO DELAY")
node2.query("DROP TABLE IF EXISTS drop_detached_test NO DELAY")
node1.query(
"""
CREATE TABLE drop_detached_test ON CLUSTER test_cluster (d UInt64)
ENGINE=ReplicatedMergeTree('/clickhouse/tables/drop_detached_test', '{}')
ORDER BY d PARTITION BY d
SETTINGS storage_policy='s3'
"""
.format('{replica}')
)
node1.query("INSERT INTO drop_detached_test VALUES (0)")
node1.query("ALTER TABLE drop_detached_test FREEZE WITH NAME 'detach_backup1'")
node1.query("INSERT INTO drop_detached_test VALUES (1)")
node1.query("ALTER TABLE drop_detached_test FREEZE WITH NAME 'detach_backup2'")
objects1 = node1.get_backuped_s3_objects("s31", "detach_backup1")
objects2 = node1.get_backuped_s3_objects("s31", "detach_backup2")
objects_diff = list(set(objects2) - set(objects1))
node1.query("ALTER TABLE drop_detached_test UNFREEZE WITH NAME 'detach_backup2'")
node1.query("ALTER TABLE drop_detached_test UNFREEZE WITH NAME 'detach_backup1'")
node1.query("ALTER TABLE drop_detached_test DETACH PARTITION '0'")
node1.query("ALTER TABLE drop_detached_test DETACH PARTITION '1'")
wait_mutations(node1, "drop_detached_test", 10)
wait_mutations(node2, "drop_detached_test", 10)
check_objects_exisis(cluster, objects1)
check_objects_exisis(cluster, objects2)
node2.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '1'", settings={"allow_drop_detached": 1})
wait_mutations(node1, "drop_detached_test", 10)
wait_mutations(node2, "drop_detached_test", 10)
check_objects_exisis(cluster, objects1)
check_objects_exisis(cluster, objects2)
node1.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '1'", settings={"allow_drop_detached": 1})
wait_mutations(node1, "drop_detached_test", 10)
wait_mutations(node2, "drop_detached_test", 10)
check_objects_exisis(cluster, objects1)
check_objects_not_exisis(cluster, objects_diff)
node1.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '0'", settings={"allow_drop_detached": 1})
wait_mutations(node1, "drop_detached_test", 10)
wait_mutations(node2, "drop_detached_test", 10)
check_objects_exisis(cluster, objects1)
node2.query("ALTER TABLE drop_detached_test DROP DETACHED PARTITION '0'", settings={"allow_drop_detached": 1})
wait_mutations(node1, "drop_detached_test", 10)
wait_mutations(node2, "drop_detached_test", 10)
check_objects_not_exisis(cluster, objects1)
def test_s3_zero_copy_concurrent_merge(cluster):
node1 = cluster.instances["node1"]
node2 = cluster.instances["node2"]

View File

@ -45,6 +45,7 @@ com
/?query=hello world+foo+bar
/?query=hello world+foo+bar
/?query=hello world+foo+bar
/?query=hello world foo+bar
/a/b/c
/a/b/c
@ -57,6 +58,7 @@ query=hello world+foo+bar
query=hello world+foo+bar
query=hello world+foo+bar
query=hello world+foo+bar
query=hello world foo+bar
====FRAGMENT====
@ -71,6 +73,7 @@ query=hello world+foo+bar#a=b
query=hello world+foo+bar#a=b
query=hello world+foo+bar#a=b
#a=b
query=hello world foo+bar#a=b
====CUT TO FIRST SIGNIFICANT SUBDOMAIN====
example.com
example.com

View File

@ -49,6 +49,7 @@ SELECT decodeURLComponent(pathFull('//127.0.0.1/?query=hello%20world+foo%2Bbar')
SELECT decodeURLComponent(pathFull('http://127.0.0.1/?query=hello%20world+foo%2Bbar')) AS Path;
SELECT decodeURLComponent(materialize(pathFull('http://127.0.0.1/?query=hello%20world+foo%2Bbar'))) AS Path;
SELECT decodeURLComponent(materialize(pathFull('//127.0.0.1/?query=hello%20world+foo%2Bbar'))) AS Path;
SELECT decodeURLFormComponent(materialize(pathFull('//127.0.0.1/?query=hello%20world+foo%2Bbar'))) AS Path;
SELECT path('http://127.0.0.1') AS Path;
SELECT path('http://127.0.0.1/a/b/c') AS Path;
SELECT path('http://127.0.0.1:443/a/b/c') AS Path;
@ -62,6 +63,7 @@ SELECT decodeURLComponent(queryString('http://127.0.0.1/?query=hello%20world+foo
SELECT decodeURLComponent(queryString('http://127.0.0.1:443/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLComponent(queryString('http://paul@127.0.0.1:443/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLComponent(queryString('//paul@127.0.0.1:443/?query=hello%20world+foo%2Bbar'));
SELECT decodeURLFormComponent(queryString('//paul@127.0.0.1:443/?query=hello%20world+foo%2Bbar'));
SELECT '====FRAGMENT====';
SELECT decodeURLComponent(fragment('http://127.0.0.1/?query=hello%20world+foo%2Bbar'));
@ -78,6 +80,7 @@ SELECT decodeURLComponent(queryStringAndFragment('http://127.0.0.1/?query=hello%
SELECT decodeURLComponent(queryStringAndFragment('http://paul@127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT decodeURLComponent(queryStringAndFragment('//paul@127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT decodeURLComponent(queryStringAndFragment('//paul@127.0.0.1/#a=b'));
SELECT decodeURLFormComponent(queryStringAndFragment('//paul@127.0.0.1/?query=hello%20world+foo%2Bbar#a=b'));
SELECT '====CUT TO FIRST SIGNIFICANT SUBDOMAIN====';
SELECT cutToFirstSignificantSubdomain('http://www.example.com');

View File

@ -1 +1 @@
SELECT if(CAST(NULL), '2.55', NULL) AS x; -- { serverError 42 }
SELECT if(CAST(NULL AS Nullable(UInt8)), '2.55', NULL) AS x;

View File

@ -6,7 +6,7 @@ SELECT
count() AS c
FROM numbers(10)
GROUP BY
arrayMap(x -> reinterpretAsUInt8(substring(randomString(randomString(range(randomString(255), NULL)), NULL))), range(3)),
arrayMap(x -> reinterpretAsUInt8(substring(randomString(randomString(range(randomString(255), NULL)), NULL), NULL)), range(3)),
randomString(range(randomString(1048577), NULL), NULL),
byte
ORDER BY byte ASC;

View File

@ -0,0 +1,14 @@
1
1
1
1
1
1
1
1
1
1
1
1
1
1

View File

@ -0,0 +1,25 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# Should finish in reasonable time (milliseconds).
# In previous versions this query led to exponential backtracking.
echo 'SELECT '"$(perl -e 'print "CAST(" x 100')"'a b c'"$(perl -e 'print ")" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'Syntax error'
echo 'SELECT '"$(perl -e 'print "CAST(" x 100')"'a, b'"$(perl -e 'print ")" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'Syntax error'
echo 'SELECT '"$(perl -e 'print "CAST(" x 100')"'a AS b'"$(perl -e 'print ")" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'Syntax error'
echo 'SELECT '"$(perl -e 'print "CAST(" x 100')"'1'"$(perl -e 'print ", '"'UInt8'"')" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000
echo 'SELECT '"$(perl -e 'print "CAST(" x 100')"'1'"$(perl -e 'print " AS UInt8)" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000
echo "SELECT fo,22222?LUTAY(SELECT(NOT CAUTAY(SELECT(NOT CAST(NOTT(NOT CAST(NOT NOT LEfT(NOT coARRAYlumnsFLuTAY(SELECT(NO0?LUTAY(SELECT(NOT CAUTAY(SELECT(NOT CAST(NOTT(NOT CAST(NOT NOT LEfT(NOT coARRAYlumnsFLuTAY(SELECT(NOTAYTAY(SELECT(NOTAYEFAULT(fo,22222?LUTAY(%SELECT(NOT CAST(NOT NOTAYTAY(SELECT(NOTAYEFAULT(fo,22222?LUTAY(SELECT(NOT CAST(NOT NOT (NOe)))))))))))))))))))))))))))))))))" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'Syntax error'
echo "SELECT position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(a b))))))))))))))))))))" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'Syntax error'
echo "SELECT position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(a, b))))))))))))))))))))" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'UNKNOWN_IDENTIFIER'
echo "SELECT position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(position(a, b, c))))))))))))))))))))" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'UNKNOWN_IDENTIFIER'
echo 'SELECT '"$(perl -e 'print "position(" x 100')"'x'"$(perl -e 'print ")" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'UNKNOWN_IDENTIFIER'
echo 'SELECT '"$(perl -e 'print "position(" x 100')"'x y'"$(perl -e 'print ")" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'Syntax error'
echo 'SELECT '"$(perl -e 'print "position(" x 100')"'x IN y'"$(perl -e 'print ")" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'UNKNOWN_IDENTIFIER'
echo 'SELECT '"$(perl -e 'print "position(" x 100')"'x'"$(perl -e 'print " IN x)" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'UNKNOWN_IDENTIFIER'
echo 'SELECT '"$(perl -e 'print "position(" x 100')"'x'"$(perl -e 'print ", x)" x 100')" | ${CLICKHOUSE_LOCAL} --max_parser_depth 10000 2>&1 | grep -cF 'UNKNOWN_IDENTIFIER'

View File

@ -69,4 +69,3 @@ WITH
toDateTime('2021-12-14 11:22:33') AS date_time_value,
toDateTime64('2021-12-14 11:22:33', 3) AS date_time_64_value
SELECT monthName(date_value), monthName(date_time_value), monthName(date_time_64_value);

View File

@ -0,0 +1,36 @@
1
[]
1
world
world
world
world
def
abc
bcde
abcdef
abcdef
abcdef
2022
Hello
3
3
2023-01-01
2023-01-01
2023-01-01
2023-01-01
2023-01-01
2023-01-01
2023-01-01
2023-01-01
2021-01-01
2021-01-01
2021-01-01
2021-01-01
2021-01-01
2021-01-01
2021-01-01
2021-01-01
1
1
1

View File

@ -0,0 +1,44 @@
SELECT CAST(1 AS UInt8);
SELECT CAST([] AS Array(UInt8));
SELECT CAST(1, 'UInt8');
SELECT SUBSTRING('Hello, world' FROM 8);
SELECT SUBSTRING('Hello, world' FROM 8 FOR 5);
SELECT SUBSTRING('Hello, world', 8);
SELECT SUBSTRING('Hello, world', 8, 5);
SELECT TRIM(LEADING 'abc' FROM 'abcdef');
SELECT TRIM(TRAILING 'def' FROM 'abcdef');
SELECT TRIM(BOTH 'af' FROM 'abcdef');
SELECT TRIM(' abcdef ');
SELECT LTRIM(' abcdef ');
SELECT RTRIM(' abcdef ');
SELECT EXTRACT(YEAR FROM DATE '2022-01-01');
SELECT EXTRACT('Hello, world', '^\w+');
SELECT POSITION('ll' IN 'Hello');
SELECT POSITION('Hello', 'll');
SELECT DATE_ADD(YEAR, 1, DATE '2022-01-01');
SELECT DATE_ADD(INTERVAL 1 YEAR, DATE '2022-01-01');
SELECT DATEADD(YEAR, 1, DATE '2022-01-01');
SELECT DATEADD(INTERVAL 1 YEAR, DATE '2022-01-01');
SELECT TIMESTAMP_ADD(YEAR, 1, DATE '2022-01-01');
SELECT TIMESTAMP_ADD(INTERVAL 1 YEAR, DATE '2022-01-01');
SELECT TIMESTAMPADD(YEAR, 1, DATE '2022-01-01');
SELECT TIMESTAMPADD(INTERVAL 1 YEAR, DATE '2022-01-01');
SELECT DATE_SUB(YEAR, 1, DATE '2022-01-01');
SELECT DATE_SUB(DATE '2022-01-01', INTERVAL 1 YEAR);
SELECT DATESUB(YEAR, 1, DATE '2022-01-01');
SELECT DATESUB(DATE '2022-01-01', INTERVAL 1 YEAR);
SELECT TIMESTAMP_SUB(YEAR, 1, DATE '2022-01-01');
SELECT TIMESTAMP_SUB(DATE '2022-01-01', INTERVAL 1 YEAR);
SELECT TIMESTAMPSUB(YEAR, 1, DATE '2022-01-01');
SELECT TIMESTAMPSUB(DATE '2022-01-01', INTERVAL 1 YEAR);
SELECT DATE_DIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01');
SELECT DATEDIFF(YEAR, DATE '2021-01-01', DATE '2022-01-01');
SELECT EXISTS (SELECT 1);

View File

@ -7,7 +7,7 @@ SELECT arrayFirst(x -> 0, [1, 2, 3]);
SELECT 'ArrayFirst non constant predicate';
SELECT arrayFirst(x -> x >= 2, emptyArrayUInt8());
SELECT arrayFirst(x -> x >= 2, [1, 2, 3]);
SELECT arrayFirst(x -> x >= 2, [1, 2, 3]);
SELECT arrayFirst(x -> x >= 2, materialize([1, 2, 3]));
SELECT 'ArrayLast constant predicate';
SELECT arrayLast(x -> 1, emptyArrayUInt8());
@ -18,4 +18,4 @@ SELECT arrayLast(x -> 0, [1, 2, 3]);
SELECT 'ArrayLast non constant predicate';
SELECT arrayLast(x -> x >= 2, emptyArrayUInt8());
SELECT arrayLast(x -> x >= 2, [1, 2, 3]);
SELECT arrayLast(x -> x >= 2, [1, 2, 3]);
SELECT arrayLast(x -> x >= 2, materialize([1, 2, 3]));

View File

@ -0,0 +1,18 @@
ArrayFirstIndex constant predicate
0
0
1
0
ArrayFirstIndex non constant predicate
0
2
2
ArrayLastIndex constant predicate
0
0
3
0
ArrayLastIndex non constant predicate
0
3
3

View File

@ -0,0 +1,21 @@
SELECT 'ArrayFirstIndex constant predicate';
SELECT arrayFirstIndex(x -> 1, emptyArrayUInt8());
SELECT arrayFirstIndex(x -> 0, emptyArrayUInt8());
SELECT arrayFirstIndex(x -> 1, [1, 2, 3]);
SELECT arrayFirstIndex(x -> 0, [1, 2, 3]);
SELECT 'ArrayFirstIndex non constant predicate';
SELECT arrayFirstIndex(x -> x >= 2, emptyArrayUInt8());
SELECT arrayFirstIndex(x -> x >= 2, [1, 2, 3]);
SELECT arrayFirstIndex(x -> x >= 2, [1, 2, 3]);
SELECT 'ArrayLastIndex constant predicate';
SELECT arrayLastIndex(x -> 1, emptyArrayUInt8());
SELECT arrayLastIndex(x -> 0, emptyArrayUInt8());
SELECT arrayLastIndex(x -> 1, [1, 2, 3]);
SELECT arrayLastIndex(x -> 0, materialize([1, 2, 3]));
SELECT 'ArrayLastIndex non constant predicate';
SELECT arrayLastIndex(x -> x >= 2, emptyArrayUInt8());
SELECT arrayLastIndex(x -> x >= 2, [1, 2, 3]);
SELECT arrayLastIndex(x -> x >= 2, materialize([1, 2, 3]));

View File

@ -0,0 +1 @@
0 1 1 Value 1

View File

@ -0,0 +1,29 @@
DROP TABLE IF EXISTS 02162_test_table;
CREATE TABLE 02162_test_table
(
id UInt64,
value String,
range_value UInt64
) ENGINE=TinyLog;
INSERT INTO 02162_test_table VALUES (0, 'Value', 1);
DROP DICTIONARY IF EXISTS 02162_test_dictionary;
CREATE DICTIONARY 02162_test_dictionary
(
id UInt64,
value String,
range_value UInt64,
start UInt64 EXPRESSION range_value,
end UInt64 EXPRESSION range_value
)
PRIMARY KEY id
SOURCE(CLICKHOUSE(TABLE '02162_test_table'))
LAYOUT(RANGE_HASHED())
RANGE(MIN start MAX end)
LIFETIME(0);
SELECT * FROM 02162_test_dictionary;
DROP DICTIONARY 02162_test_dictionary;
DROP TABLE 02162_test_table;

View File

@ -0,0 +1 @@
1

View File

@ -0,0 +1,2 @@
WITH 2 AS `b.c`, [4, 5] AS a, 6 AS u, 3 AS v, 2 AS d, TRUE AS e, 1 AS f, 0 AS g, 2 AS h, 'Hello' AS i, 'World' AS j, TIMESTAMP '2022-02-02 02:02:02' AS w, [] AS k, (1, 2) AS l, 2 AS m, 3 AS n, [] AS o, [1] AS p, 1 AS q, q AS r, 1 AS s, 1 AS t
SELECT INTERVAL CASE CASE WHEN NOT -a[b.c] * u DIV v + d IS NOT NULL AND e OR f BETWEEN g AND h THEN i ELSE j END WHEN w THEN k END || [l, (m, n)] MINUTE IS NULL OR NOT o::Array(INT) = p <> q < r > s != t AS upyachka;

View File

@ -0,0 +1,129 @@
#!/usr/bin/env python3
import argparse
import socket
import uuid
from kazoo.client import KazooClient
def parse_args():
"""
Parse command-line arguments.
"""
parser = argparse.ArgumentParser()
parser.add_argument('--hosts', default=socket.getfqdn() + ':2181', help='ZooKeeper hosts (host:port,host:port,...)')
parser.add_argument('-s', '--secure', default=False, action='store_true', help='Use secure connection')
parser.add_argument('--cert', default='', help='Client TLS certificate file')
parser.add_argument('--key', default='', help='Client TLS key file')
parser.add_argument('--ca', default='', help='Client TLS ca file')
parser.add_argument('-u', '--user', default='', help='ZooKeeper ACL user')
parser.add_argument('-p', '--password', default='', help='ZooKeeper ACL password')
parser.add_argument('-r', '--root', default='/clickhouse', help='ZooKeeper root path for ClickHouse')
parser.add_argument('-z', '--zcroot', default='zero_copy', help='ZooKeeper node for new zero-copy data')
parser.add_argument('--dryrun', default=False, action='store_true', help='Do not perform any actions')
parser.add_argument('--cleanup', default=False, action='store_true', help='Clean old nodes')
parser.add_argument('-v', '--verbose', action='store_true', default=False, help='Verbose mode')
return parser.parse_args()
# Several folders to heuristic that zookeepr node is folder node
# May be false positive when someone creates set of tables with same paths
table_nodes = ['alter_partition_version', 'block_numbers', 'blocks', 'columns', 'leader_election']
zc_nodes = ['zero_copy_s3', 'zero_copy_hdfs']
def convert_node(client, args, path, zc_node):
base_path = f'{path}/{zc_node}/shared'
parts = client.get_children(base_path)
table_id_path = f'{path}/table_id'
table_id = ''
if client.exists(table_id_path):
table_id = client.get(table_id_path)[0].decode('UTF-8')
else:
table_id = str(uuid.uuid4())
if args.verbose:
print(f'Make table_id "{table_id_path}" = "{table_id}"')
if not args.dryrun:
client.create(table_id_path, bytes(table_id, 'UTF-8'))
for part in parts:
part_path = f'{base_path}/{part}'
uniq_ids = client.get_children(part_path)
for uniq_id in uniq_ids:
uniq_path = f'{part_path}/{uniq_id}'
replicas = client.get_children(uniq_path)
for replica in replicas:
replica_path = f'{uniq_path}/{replica}'
new_path = f'{args.root}/{args.zcroot}/{zc_node}/{table_id}/{part}/{uniq_id}/{replica}'
if not client.exists(new_path):
if args.verbose:
print(f'Make node "{new_path}"')
if not args.dryrun:
client.ensure_path(f'{args.root}/{args.zcroot}/{zc_node}/{table_id}/{part}/{uniq_id}')
client.create(new_path, value=b'lock')
if args.cleanup:
if args.verbose:
print(f'Remove node "{replica_path}"')
if not args.dryrun:
client.delete(replica_path)
if args.cleanup and not args.dryrun:
client.delete(uniq_path)
if args.cleanup and not args.dryrun:
client.delete(part_path)
if args.cleanup and not args.dryrun:
client.delete(base_path)
client.delete(f'{path}/{zc_node}')
def convert_table(client, args, path, nodes):
print(f'Convert table nodes by path "{path}"')
for zc_node in zc_nodes:
if zc_node in nodes:
convert_node(client, args, path, zc_node)
def is_like_a_table(nodes):
for tn in table_nodes:
if tn not in nodes:
return False
return True
def scan_recursive(client, args, path):
nodes = client.get_children(path)
if is_like_a_table(nodes):
convert_table(client, args, path, nodes)
else:
for node in nodes:
scan_recursive(client, args, f'{path}/{node}')
def scan(client, args):
nodes = client.get_children(args.root)
for node in nodes:
if node != args.zcroot:
scan_recursive(client, args, f'{args.root}/{node}')
def get_client(args):
client = KazooClient(connection_retry=3,
command_retry=3,
timeout=1,
hosts=args.hosts,
use_ssl=args.secure,
certfile=args.cert,
keyfile=args.key,
ca=args.ca
)
client.start()
if (args.user and args.password):
client.add_auth('digest', f'{args.user}:{args.password}')
return client
def main():
args = parse_args()
client = get_client(args)
scan(client, args)
if __name__ == '__main__':
main()