mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
remove separate getTableName() and getDatabaseName()
This commit is contained in:
parent
ef129b4b7c
commit
42296cc80a
@ -47,7 +47,8 @@ std::ostream & operator<<(std::ostream & stream, const IDataType & what)
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const IStorage & what)
|
||||
{
|
||||
stream << "IStorage(name = " << what.getName() << ", tableName = " << what.getTableName() << ") {"
|
||||
auto table_id = what.getStorageID();
|
||||
stream << "IStorage(name = " << what.getName() << ", tableName = " << table_id.table_name << ") {"
|
||||
<< what.getColumns().getAllPhysical().toString() << "}";
|
||||
return stream;
|
||||
}
|
||||
|
@ -54,10 +54,11 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
if (auto * materialized_view = dynamic_cast<const StorageMaterializedView *>(dependent_table.get()))
|
||||
{
|
||||
StoragePtr inner_table = materialized_view->getTargetTable();
|
||||
auto inner_table_id = inner_table->getStorageID();
|
||||
query = materialized_view->getInnerQuery();
|
||||
std::unique_ptr<ASTInsertQuery> insert = std::make_unique<ASTInsertQuery>();
|
||||
insert->database = inner_table->getDatabaseName();
|
||||
insert->table = inner_table->getTableName();
|
||||
insert->database = inner_table_id.database_name;
|
||||
insert->table = inner_table_id.table_name; //FIXME add uuid
|
||||
ASTPtr insert_query_ptr(insert.release());
|
||||
InterpreterInsertQuery interpreter(insert_query_ptr, *views_context);
|
||||
BlockIO io = interpreter.execute();
|
||||
|
@ -132,9 +132,9 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr
|
||||
|
||||
{
|
||||
/// init create query.
|
||||
|
||||
create_table_query->table = storage->getTableName();
|
||||
create_table_query->database = storage->getDatabaseName();
|
||||
auto table_id = storage->getStorageID();
|
||||
create_table_query->table = table_id.table_name;
|
||||
create_table_query->database = table_id.database_name;
|
||||
|
||||
for (const auto & column_type_and_name : storage->getColumns().getOrdinary())
|
||||
{
|
||||
@ -144,7 +144,7 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr
|
||||
columns_expression_list->children.emplace_back(column_declaration);
|
||||
}
|
||||
|
||||
auto mysql_table_name = std::make_shared<ASTLiteral>(storage->getTableName());
|
||||
auto mysql_table_name = std::make_shared<ASTLiteral>(table_id.table_name);
|
||||
auto storage_engine_arguments = table_storage_define->as<ASTStorage>()->engine->arguments;
|
||||
storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name);
|
||||
}
|
||||
|
@ -79,13 +79,14 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
|
||||
if (database_and_table.first && database_and_table.second)
|
||||
{
|
||||
auto table_id = database_and_table.second->getStorageID();
|
||||
if (kind == ASTDropQuery::Kind::Detach)
|
||||
{
|
||||
database_and_table.second->shutdown();
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto table_lock = database_and_table.second->lockExclusively(context.getCurrentQueryId());
|
||||
/// Drop table from memory, don't touch data and metadata
|
||||
database_and_table.first->detachTable(database_and_table.second->getTableName());
|
||||
database_and_table.first->detachTable(table_id.table_name);
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Truncate)
|
||||
{
|
||||
@ -107,7 +108,7 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
|
||||
const std::string metadata_file_without_extension =
|
||||
database_and_table.first->getMetadataPath()
|
||||
+ escapeForFileName(database_and_table.second->getTableName());
|
||||
+ escapeForFileName(table_id.table_name);
|
||||
|
||||
const auto prev_metadata_name = metadata_file_without_extension + ".sql";
|
||||
const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop";
|
||||
@ -131,7 +132,7 @@ BlockIO InterpreterDropQuery::executeToTable(
|
||||
String table_data_path_relative = database_and_table.first->getTableDataPath(table_name);
|
||||
|
||||
/// Delete table metadata and table itself from memory
|
||||
database_and_table.first->removeTable(context, database_and_table.second->getTableName());
|
||||
database_and_table.first->removeTable(context, table_id.table_name);
|
||||
database_and_table.second->is_dropped = true;
|
||||
|
||||
/// If it is not virtual database like Dictionary then drop remaining data dir
|
||||
|
@ -295,7 +295,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
if (auto view_source = context->getViewSource())
|
||||
{
|
||||
auto & storage_values = static_cast<const StorageValues &>(*view_source);
|
||||
if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name)
|
||||
auto tmp_table_id = storage_values.getStorageID();
|
||||
if (tmp_table_id.database_name == database_name && tmp_table_id.table_name == table_name)
|
||||
{
|
||||
/// Read from view source.
|
||||
storage = context->getViewSource();
|
||||
@ -419,9 +420,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
sanitizeBlock(result_header);
|
||||
|
||||
/// Remove limits for some tables in the `system` database.
|
||||
if (storage && (storage->getDatabaseName() == "system"))
|
||||
if (storage && (table_id->database_name == "system"))
|
||||
{
|
||||
String table_name = storage->getTableName();
|
||||
String table_name = table_id->table_name;
|
||||
if ((table_name == "quotas") || (table_name == "quota_usage") || (table_name == "one"))
|
||||
{
|
||||
options.ignore_quota = true;
|
||||
|
@ -110,7 +110,8 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP
|
||||
|
||||
try
|
||||
{
|
||||
part_log = current_context.getPartLog(parts.front()->storage.getDatabaseName()); // assume parts belong to the same table
|
||||
auto table_id = parts.front()->storage.getStorageID();
|
||||
part_log = current_context.getPartLog(table_id.database_name); // assume parts belong to the same table
|
||||
if (!part_log)
|
||||
return false;
|
||||
|
||||
@ -122,8 +123,8 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP
|
||||
elem.event_time = time(nullptr);
|
||||
elem.duration_ms = elapsed_ns / 1000000;
|
||||
|
||||
elem.database_name = part->storage.getDatabaseName();
|
||||
elem.table_name = part->storage.getTableName();
|
||||
elem.database_name = table_id.database_name;
|
||||
elem.table_name = table_id.table_name;
|
||||
elem.partition_id = part->info.partition_id;
|
||||
elem.part_name = part->name;
|
||||
elem.path_on_disk = part->getFullPath();
|
||||
|
@ -90,7 +90,7 @@ void DistributedBlockOutputStream::write(const Block & block)
|
||||
if (ordinary_block.has(col.name))
|
||||
{
|
||||
ordinary_block.erase(col.name);
|
||||
LOG_DEBUG(log, storage.getTableName()
|
||||
LOG_DEBUG(log, storage.getStorageID().getNameForLogs()
|
||||
<< ": column " + col.name + " will be removed, "
|
||||
<< "because it is MATERIALIZED");
|
||||
}
|
||||
@ -515,7 +515,7 @@ void DistributedBlockOutputStream::writeAsyncImpl(const Block & block, const siz
|
||||
else
|
||||
{
|
||||
if (shard_info.dir_name_for_internal_replication.empty())
|
||||
throw Exception("Directory name for async inserts is empty, table " + storage.getTableName(), ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Directory name for async inserts is empty, table " + storage.getStorageID().getNameForLogs(), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
writeToShard(block, {shard_info.dir_name_for_internal_replication});
|
||||
}
|
||||
|
@ -176,7 +176,7 @@ void IStorage::check(const Names & column_names, bool include_virtuals) const
|
||||
{
|
||||
if (columns_map.end() == columns_map.find(name))
|
||||
throw Exception(
|
||||
"There is no column with name " + backQuote(name) + " in table " + getTableName() + ". There are columns: " + list_of_columns,
|
||||
"There is no column with name " + backQuote(name) + " in table " + getStorageID().getNameForLogs() + ". There are columns: " + list_of_columns,
|
||||
ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
if (unique_names.end() != unique_names.find(name))
|
||||
@ -344,7 +344,7 @@ TableStructureWriteLockHolder IStorage::lockAlterIntention(const String & query_
|
||||
void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id)
|
||||
{
|
||||
if (!lock_holder.alter_intention_lock)
|
||||
throw Exception("Alter intention lock for table " + getTableName() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id);
|
||||
}
|
||||
@ -352,7 +352,7 @@ void IStorage::lockNewDataStructureExclusively(TableStructureWriteLockHolder & l
|
||||
void IStorage::lockStructureExclusively(TableStructureWriteLockHolder & lock_holder, const String & query_id)
|
||||
{
|
||||
if (!lock_holder.alter_intention_lock)
|
||||
throw Exception("Alter intention lock for table " + getTableName() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Alter intention lock for table " + getStorageID().getNameForLogs() + " was not taken. This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!lock_holder.new_data_structure_lock)
|
||||
lock_holder.new_data_structure_lock = new_data_structure_lock->getLock(RWLockImpl::Write, query_id);
|
||||
@ -405,15 +405,14 @@ void IStorage::alter(
|
||||
if (params.isModifyingData())
|
||||
throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
const String database_name = getDatabaseName();
|
||||
const String table_name = getTableName();
|
||||
auto table_id = getStorageID();
|
||||
|
||||
if (params.isSettingsAlter())
|
||||
{
|
||||
SettingsChanges new_changes;
|
||||
params.applyForSettingsOnly(new_changes);
|
||||
IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, getColumns(), getIndices(), getConstraints(), settings_modifier);
|
||||
context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier); //FIXME
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -422,7 +421,7 @@ void IStorage::alter(
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.applyForColumnsOnly(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, new_constraints, {});
|
||||
context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
}
|
||||
|
@ -90,18 +90,6 @@ public:
|
||||
/// The name of the table.
|
||||
StorageID getStorageID(std::unique_lock<std::mutex> * lock = nullptr) const;
|
||||
|
||||
// FIXME remove those methods
|
||||
std::string getTableName() const
|
||||
{
|
||||
std::lock_guard lock(id_mutex);
|
||||
return id.table_name;
|
||||
}
|
||||
std::string getDatabaseName() const
|
||||
{
|
||||
std::lock_guard lock(id_mutex);
|
||||
return id.database_name;
|
||||
}
|
||||
|
||||
/// Returns true if the storage receives data from a remote server or servers.
|
||||
virtual bool isRemote() const { return false; }
|
||||
|
||||
|
@ -9,7 +9,7 @@ class ProxyStorage : public IStorage
|
||||
{
|
||||
public:
|
||||
ProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_)
|
||||
: IStorage({"", storage_->getTableName()}), storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {}
|
||||
: IStorage({"", storage_->getStorageID().table_name}), storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {}
|
||||
|
||||
public:
|
||||
std::string getName() const override { return "ProxyStorage(" + storage->getName() + ")"; }
|
||||
|
@ -49,11 +49,7 @@ public:
|
||||
|
||||
protected:
|
||||
StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_)
|
||||
: IStorage({
|
||||
part_->storage.getDatabaseName(),
|
||||
part_->storage.getTableName() + " (part " + part_->name + ")"
|
||||
}
|
||||
, part_->storage.getVirtuals())
|
||||
: IStorage(getIDFromPart(part_), part_->storage.getVirtuals())
|
||||
, part(part_)
|
||||
{
|
||||
setColumns(part_->storage.getColumns());
|
||||
@ -62,6 +58,12 @@ protected:
|
||||
|
||||
private:
|
||||
MergeTreeData::DataPartPtr part;
|
||||
|
||||
static StorageID getIDFromPart(const MergeTreeData::DataPartPtr & part_)
|
||||
{
|
||||
auto table_id = part_->storage.getStorageID();
|
||||
return StorageID(table_id.database_name, table_id.table_name + " (part " + part_->name + ")");
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -713,8 +713,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context,
|
||||
{
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
const String database_name_ = getDatabaseName();
|
||||
const String table_name_ = getTableName();
|
||||
auto table_id = getStorageID();
|
||||
|
||||
/// So that no blocks of the old structure remain.
|
||||
optimize({} /*query*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/, context);
|
||||
@ -723,7 +722,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context,
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.applyForColumnsOnly(new_columns);
|
||||
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, new_constraints, {});
|
||||
context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {}); //FIXME
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -402,14 +402,13 @@ void StorageDistributed::alter(
|
||||
{
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
const String current_database_name = getDatabaseName();
|
||||
const String current_table_name = getTableName();
|
||||
auto table_id = getStorageID();
|
||||
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
auto new_constraints = getConstraints();
|
||||
params.applyForColumnsOnly(new_columns);
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, {});
|
||||
context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -281,7 +281,10 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
|
||||
SelectQueryInfo modified_query_info = query_info;
|
||||
modified_query_info.query = query_info.query->clone();
|
||||
|
||||
VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", storage ? storage->getTableName() : "");
|
||||
StorageID table_id;
|
||||
if (storage)
|
||||
table_id = storage->getStorageID();
|
||||
VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_id.table_name);
|
||||
|
||||
if (!storage)
|
||||
return BlockInputStreams{
|
||||
@ -301,7 +304,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
|
||||
}
|
||||
else if (processed_stage > storage->getQueryProcessingStage(modified_context))
|
||||
{
|
||||
modified_query_info.query->as<ASTSelectQuery>()->replaceDatabaseAndTable(source_database, storage->getTableName());
|
||||
modified_query_info.query->as<ASTSelectQuery>()->replaceDatabaseAndTable(source_database, table_id.table_name);
|
||||
|
||||
/// Maximum permissible parallelism is streams_num
|
||||
modified_context.getSettingsRef().max_threads = UInt64(streams_num);
|
||||
@ -332,7 +335,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
|
||||
{
|
||||
if (has_table_virtual_column)
|
||||
source_stream = std::make_shared<AddingConstColumnBlockInputStream<String>>(
|
||||
source_stream, std::make_shared<DataTypeString>(), storage->getTableName(), "_table");
|
||||
source_stream, std::make_shared<DataTypeString>(), table_id.table_name, "_table");
|
||||
|
||||
/// Subordinary tables could have different but convertible types, like numeric types of different width.
|
||||
/// We must return streams with structure equals to structure of Merge table.
|
||||
@ -381,7 +384,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
|
||||
if (storage.get() != this)
|
||||
{
|
||||
selected_tables.emplace_back(storage, get_lock ? storage->lockStructureForShare(false, query_id) : TableStructureReadLockHolder{});
|
||||
virtual_column->insert(storage->getTableName());
|
||||
auto table_id = storage->getStorageID();
|
||||
virtual_column->insert(table_id.table_name);
|
||||
}
|
||||
|
||||
iterator->next();
|
||||
@ -394,7 +398,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
|
||||
auto values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_table");
|
||||
|
||||
/// Remove unused tables from the list
|
||||
selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getTableName()) == values.end(); });
|
||||
//FIXME table name can be changed, use StorageID
|
||||
selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getStorageID().table_name) == values.end(); });
|
||||
}
|
||||
|
||||
return selected_tables;
|
||||
|
@ -260,8 +260,7 @@ void StorageMergeTree::alter(
|
||||
const Context & context,
|
||||
TableStructureWriteLockHolder & table_lock_holder)
|
||||
{
|
||||
const String current_database_name = getDatabaseName();
|
||||
const String current_table_name = getTableName();
|
||||
auto table_id = getStorageID();
|
||||
|
||||
lockNewDataStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
@ -313,7 +312,7 @@ void StorageMergeTree::alter(
|
||||
{
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
|
||||
update_metadata();
|
||||
}
|
||||
@ -330,7 +329,7 @@ void StorageMergeTree::alter(
|
||||
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
context.getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, new_columns, new_indices, new_constraints, storage_modifier);
|
||||
|
||||
update_metadata();
|
||||
|
||||
|
@ -3249,8 +3249,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
LOG_DEBUG(log, "Doing ALTER");
|
||||
|
||||
const String current_database_name = getDatabaseName();
|
||||
const String current_table_name = getTableName();
|
||||
auto table_id = getStorageID();
|
||||
|
||||
/// We cannot check this alter commands with method isModifyingData()
|
||||
/// because ReplicatedMergeTree stores both columns and metadata for
|
||||
@ -3267,8 +3266,8 @@ void StorageReplicatedMergeTree::alter(
|
||||
changeSettings(new_changes, table_lock_holder);
|
||||
|
||||
IDatabase::ASTModifier settings_modifier = getSettingsModifier(new_changes);
|
||||
global_context.getDatabase(current_database_name)->alterTable(
|
||||
query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier);
|
||||
global_context.getDatabase(table_id.database_name)->alterTable(
|
||||
query_context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -3349,8 +3348,8 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
changeSettings(new_changes, table_lock_holder);
|
||||
|
||||
global_context.getDatabase(current_database_name)->alterTable(
|
||||
query_context, current_table_name, getColumns(), getIndices(), getConstraints(), settings_modifier);
|
||||
global_context.getDatabase(table_id.database_name)->alterTable(
|
||||
query_context, table_id.table_name, getColumns(), getIndices(), getConstraints(), settings_modifier);
|
||||
|
||||
}
|
||||
|
||||
|
@ -48,20 +48,22 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context &
|
||||
{
|
||||
const String & config_name = table_data->merging_params.graphite_params.config_name;
|
||||
|
||||
auto table_id = table_data->getStorageID();
|
||||
if (!graphite_configs.count(config_name))
|
||||
{
|
||||
//TODO add uuid
|
||||
Config new_config =
|
||||
{
|
||||
table_data->merging_params.graphite_params,
|
||||
{ table_data->getDatabaseName() },
|
||||
{ table_data->getTableName() },
|
||||
{ table_id.database_name },
|
||||
{ table_id.database_name },
|
||||
};
|
||||
graphite_configs.emplace(config_name, new_config);
|
||||
}
|
||||
else
|
||||
{
|
||||
graphite_configs[config_name].databases.emplace_back(table_data->getDatabaseName());
|
||||
graphite_configs[config_name].tables.emplace_back(table_data->getTableName());
|
||||
graphite_configs[config_name].databases.emplace_back(table_id.database_name);
|
||||
graphite_configs[config_name].tables.emplace_back(table_id.database_name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user