Less lambdas

This commit is contained in:
kssenii 2021-03-20 13:13:30 +00:00
parent 60fbeb8354
commit 4c37ff3fbc
5 changed files with 24 additions and 15 deletions

View File

@ -20,6 +20,8 @@ struct PostgreSQLTableStructure
std::shared_ptr<NamesAndTypesList> primary_key_columns;
};
using PostgreSQLTableStructurePtr = std::unique_ptr<PostgreSQLTableStructure>;
PostgreSQLTableStructure fetchPostgreSQLTableStructure(
PostgreSQLConnection::ConnectionPtr connection, const String & postgres_table_name, bool use_nulls, bool with_primary_key = false);

View File

@ -167,7 +167,7 @@ NameSet PostgreSQLReplicationHandler::loadFromSnapshot(std::string & snapshot_na
std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name);
tx->exec(query_str);
storage_data.second->createNestedIfNeeded([&]() { return fetchTableStructure(tx, table_name); });
storage_data.second->createNestedIfNeeded(fetchTableStructure(tx, table_name));
auto nested_storage = storage_data.second->getNested();
/// Load from snapshot, which will show table state before creation of replication slot.
@ -392,11 +392,14 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(PostgreSQLConne
}
PostgreSQLTableStructure PostgreSQLReplicationHandler::fetchTableStructure(
PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure(
std::shared_ptr<pqxx::work> tx, const std::string & table_name)
{
if (!is_postgresql_replica_database_engine)
return nullptr;
auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls;
return fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true);
return std::make_unique<PostgreSQLTableStructure>(fetchPostgreSQLTableStructure(tx, table_name, use_nulls, true));
}

View File

@ -44,8 +44,6 @@ public:
NameSet fetchRequiredTables(PostgreSQLConnection::ConnectionPtr connection_);
PostgreSQLTableStructure fetchTableStructure(std::shared_ptr<pqxx::work> tx, const std::string & table_name);
private:
using NontransactionPtr = std::shared_ptr<pqxx::nontransaction>;
using Storages = std::unordered_map<String, StorageMaterializePostgreSQL *>;
@ -74,6 +72,8 @@ private:
std::unordered_map<Int32, String> reloadFromSnapshot(const std::vector<std::pair<Int32, String>> & relation_data);
PostgreSQLTableStructurePtr fetchTableStructure(std::shared_ptr<pqxx::work> tx, const std::string & table_name);
Poco::Logger * log;
const Context & context;
const std::string database_name, connection_str, metadata_path;

View File

@ -153,7 +153,7 @@ ASTPtr StorageMaterializePostgreSQL::getColumnDeclaration(const DataTypePtr & da
/// For single storage MaterializePostgreSQL get columns and primary key columns from storage definition.
/// For database engine MaterializePostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same
/// transaction with snapshot, which is used for initial tables dump.
ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::function<PostgreSQLTableStructure()> & fetch_table_structure)
ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure)
{
auto create_table_query = std::make_shared<ASTCreateQuery>();
@ -175,9 +175,13 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::functi
}
else
{
auto table_structure = fetch_table_structure();
if (!table_structure)
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"No table structure returned for table {}.{}", table_id.database_name, table_id.table_name);
}
if (!table_structure.columns)
if (!table_structure->columns)
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"No columns returned for table {}.{}", table_id.database_name, table_id.table_name);
@ -185,17 +189,17 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::functi
StorageInMemoryMetadata storage_metadata;
ordinary_columns_and_types = *table_structure.columns;
ordinary_columns_and_types = *table_structure->columns;
storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types));
setInMemoryMetadata(storage_metadata);
if (!table_structure.primary_key_columns)
if (!table_structure->primary_key_columns)
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"No primary key columns returned for table {}.{}", table_id.database_name, table_id.table_name);
}
auto primary_key_columns = *table_structure.primary_key_columns;
auto primary_key_columns = *table_structure->primary_key_columns;
order_by_expression->name = "tuple";
order_by_expression->arguments = std::make_shared<ASTExpressionList>();
@ -238,7 +242,7 @@ ASTPtr StorageMaterializePostgreSQL::getCreateNestedTableQuery(const std::functi
}
void StorageMaterializePostgreSQL::createNestedIfNeeded(const std::function<PostgreSQLTableStructure()> & fetch_table_structure)
void StorageMaterializePostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure)
{
if (nested_loaded)
{
@ -249,7 +253,7 @@ void StorageMaterializePostgreSQL::createNestedIfNeeded(const std::function<Post
}
auto context = makeNestedTableContext();
const auto ast_create = getCreateNestedTableQuery(fetch_table_structure);
const auto ast_create = getCreateNestedTableQuery(std::move(table_structure));
try
{

View File

@ -53,7 +53,7 @@ public:
/// Called right after shutdown() in case of drop query
void shutdownFinal();
void createNestedIfNeeded(const std::function<PostgreSQLTableStructure()> & fetch_table_structure);
void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure);
/// Can be nullptr
StoragePtr tryGetNested();
@ -85,7 +85,7 @@ private:
ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const;
ASTPtr getCreateNestedTableQuery(const std::function<PostgreSQLTableStructure()> & fetch_table_structure);
ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure);
std::string getNestedTableName() const;