mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Better dependent name for nested in case of single storage
This commit is contained in:
parent
027f67affb
commit
c968ccb391
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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...")
|
||||
|
Loading…
Reference in New Issue
Block a user