Better dependent name for nested in case of single storage

This commit is contained in:
kssenii 2021-04-11 17:19:20 +00:00
parent 027f67affb
commit c968ccb391
3 changed files with 44 additions and 43 deletions

View File

@ -35,7 +35,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
static const auto NESTED_STORAGE_SUFFIX = "_ReplacingMergeTree";
static const auto NESTED_TABLE_SUFFIX = "_nested";
StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(
@ -55,6 +55,9 @@ StorageMaterializePostgreSQL::StorageMaterializePostgreSQL(
, nested_table_id(StorageID(table_id_.database_name, getNestedTableName()))
, nested_context(makeNestedTableContext(context_->getGlobalContext()))
{
if (table_id_.uuid == UUIDHelpers::Nil)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializePostgreSQL is allowed only for Atomic database");
setInMemoryMetadata(storage_metadata);
auto metadata_path = DatabaseCatalog::instance().getDatabase(getStorageID().database_name)->getMetadataPath()
@ -96,12 +99,12 @@ StoragePtr StorageMaterializePostgreSQL::tryGetNested() const
std::string StorageMaterializePostgreSQL::getNestedTableName() const
{
auto table_name = getStorageID().table_name;
auto table_id = getStorageID();
if (!is_materialize_postgresql_database)
table_name += NESTED_STORAGE_SUFFIX;
if (is_materialize_postgresql_database)
return table_id.table_name;
return table_name;
return toString(table_id.uuid) + NESTED_TABLE_SUFFIX;
}
@ -287,7 +290,7 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructure
}
std::shared_ptr<Context> StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context) const
std::shared_ptr<Context> StorageMaterializePostgreSQL::makeNestedTableContext(ContextPtr from_context)
{
auto new_context = Context::createCopy(from_context);
new_context->makeQueryContext();
@ -360,42 +363,6 @@ Pipe StorageMaterializePostgreSQL::read(
}
//void StorageMaterializePostgreSQL::renameInMemory(const StorageID & new_table_id)
//{
// auto old_table_id = getStorageID();
// auto metadata_snapshot = getInMemoryMetadataPtr();
//
// IStorage::renameInMemory(new_table_id);
// auto nested_table = tryGetNested();
//
// if (nested_table)
// {
// auto new_nested_table_name = getNestedTableName();
// auto rename = std::make_shared<ASTRenameQuery>();
//
// ASTRenameQuery::Table from;
// from.database = nested_table_id.database_name;
// from.table = nested_table_id.table_name;
//
// ASTRenameQuery::Table to;
// to.database = nested_table_id.database_name;
// to.table = new_nested_table_name;
//
// ASTRenameQuery::Element elem;
// elem.from = from;
// elem.to = to;
// rename->elements.emplace_back(elem);
//
// InterpreterRenameQuery(rename, getContext()).execute();
// target_table_id.table_name = new_target_table_name;
// }
//
// const auto & select_query = metadata_snapshot->getSelectQuery();
// // TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated
// DatabaseCatalog::instance().updateDependency(select_query.select_table_id, old_table_id, select_query.select_table_id, getStorageID());
//}
void registerStorageMaterializePostgreSQL(StorageFactory & factory)
{
auto creator_fn = [](const StorageFactory::Arguments & args)

View File

@ -65,6 +65,8 @@ public:
void setStorageMetadata();
void renameNested();
protected:
StorageMaterializePostgreSQL(
const StorageID & table_id_,
@ -85,7 +87,7 @@ private:
std::string getNestedTableName() const;
std::shared_ptr<Context> makeNestedTableContext(ContextPtr from_context) const;
static std::shared_ptr<Context> makeNestedTableContext(ContextPtr from_context);
std::string remote_table_name;
std::unique_ptr<MaterializePostgreSQLSettings> replication_settings;

View File

@ -430,6 +430,38 @@ def test_clickhouse_restart(started_cluster):
assert(int(result) == 100050)
def test_rename_table(started_cluster):
conn = get_postgres_conn(True)
cursor = conn.cursor()
create_postgres_table(cursor, 'postgresql_replica');
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)")
instance.query('''
CREATE TABLE test.postgresql_replica (key UInt64, value UInt64, _sign Int8 MATERIALIZED 1, _version UInt64 MATERIALIZED 1)
ENGINE = MaterializePostgreSQL(
'postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword')
PRIMARY KEY key; ''')
result = instance.query('SELECT count() FROM test.postgresql_replica;')
while int(result) != 50:
time.sleep(0.5)
result = instance.query('SELECT count() FROM test.postgresql_replica;')
instance.query('RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed')
result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;')
while int(result) != 50:
time.sleep(0.5)
result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;')
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50, 50)")
result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;')
while int(result) != 100:
time.sleep(0.5)
result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;')
if __name__ == '__main__':
cluster.start()
input("Cluster created, press any key to destroy...")