ClickHouse/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

590 lines
22 KiB
C++
Raw Normal View History

2021-06-27 19:09:17 +00:00
#include "StorageMaterializedPostgreSQL.h"
2021-02-18 06:06:37 +00:00
#if USE_LIBPQXX
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
2021-12-15 21:38:46 +00:00
2021-02-06 12:28:42 +00:00
#include <Common/Macros.h>
#include <Common/parseAddress.h>
#include <Common/assert_cast.h>
2021-12-15 21:38:46 +00:00
2021-09-03 11:16:32 +00:00
#include <Core/Settings.h>
2021-12-15 21:38:46 +00:00
#include <Core/PostgreSQL/Connection.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeArray.h>
2021-02-12 10:05:13 +00:00
#include <DataTypes/DataTypesDecimal.h>
2021-12-14 13:53:47 +00:00
#include <DataTypes/DataTypeFactory.h>
2021-12-15 21:38:46 +00:00
#include <Formats/FormatFactory.h>
#include <Formats/FormatSettings.h>
2021-12-15 21:38:46 +00:00
2022-05-20 19:49:31 +00:00
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTTablesInSelectQuery.h>
2021-12-15 21:38:46 +00:00
#include <Interpreters/applyTableOverride.h>
#include <Interpreters/InterpreterDropQuery.h>
2021-12-15 21:38:46 +00:00
#include <Storages/StorageFactory.h>
#include <Storages/ReadFinalForExternalReplicaStorage.h>
2021-09-03 11:16:32 +00:00
#include <Storages/StoragePostgreSQL.h>
2021-12-15 21:38:46 +00:00
#include <QueryPipeline/Pipe.h>
namespace DB
{
namespace ErrorCodes
{
2021-02-08 23:23:51 +00:00
extern const int LOGICAL_ERROR;
2021-02-13 20:46:52 +00:00
extern const int BAD_ARGUMENTS;
}
static const auto NESTED_TABLE_SUFFIX = "_nested";
2021-05-01 11:49:45 +00:00
static const auto TMP_SUFFIX = "_tmp";
2021-02-03 16:13:18 +00:00
2021-05-03 09:52:13 +00:00
/// For the case of single storage.
2021-06-27 19:09:17 +00:00
StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(
const StorageID & table_id_,
bool is_attach_,
const String & remote_database_name,
const String & remote_table_name_,
2021-04-09 10:22:06 +00:00
const postgres::ConnectionInfo & connection_info,
const StorageInMemoryMetadata & storage_metadata,
2021-04-11 07:44:40 +00:00
ContextPtr context_,
2021-06-27 19:09:17 +00:00
std::unique_ptr<MaterializedPostgreSQLSettings> replication_settings)
: IStorage(table_id_)
2021-04-11 07:44:40 +00:00
, WithContext(context_->getGlobalContext())
2021-08-29 08:50:03 +00:00
, log(&Poco::Logger::get("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(remote_database_name, remote_table_name_) + ")"))
2021-06-27 19:09:17 +00:00
, is_materialized_postgresql_database(false)
2021-05-03 09:52:13 +00:00
, has_nested(false)
2021-04-11 09:54:10 +00:00
, nested_context(makeNestedTableContext(context_->getGlobalContext()))
, nested_table_id(StorageID(table_id_.database_name, getNestedTableName()))
, remote_table_name(remote_table_name_)
, is_attach(is_attach_)
{
if (table_id_.uuid == UUIDHelpers::Nil)
2021-06-27 19:09:17 +00:00
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializedPostgreSQL is allowed only for Atomic database");
setInMemoryMetadata(storage_metadata);
2021-02-08 23:23:51 +00:00
String replication_identifier = remote_database_name + "_" + remote_table_name_;
2021-09-04 10:07:59 +00:00
replication_settings->materialized_postgresql_tables_list = remote_table_name_;
replication_handler = std::make_unique<PostgreSQLReplicationHandler>(
replication_identifier,
remote_database_name,
2021-05-01 11:49:45 +00:00
table_id_.database_name,
2021-04-09 10:22:06 +00:00
connection_info,
2021-04-11 07:44:40 +00:00
getContext(),
2021-07-04 14:56:31 +00:00
is_attach,
2021-09-04 10:07:59 +00:00
*replication_settings,
/* is_materialized_postgresql_database */false);
2021-08-31 19:24:58 +00:00
2021-12-25 14:10:21 +00:00
replication_handler->addStorage(remote_table_name, this);
replication_handler->startup(/* delayed */is_attach);
}
2021-05-03 09:52:13 +00:00
/// For the case of MaterializePosgreSQL database engine.
/// It is used when nested ReplacingMergeeTree table has not yet be created by replication thread.
/// In this case this storage can't be used for read queries.
2021-08-27 06:30:21 +00:00
StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(
const StorageID & table_id_,
ContextPtr context_,
const String & postgres_database_name,
const String & postgres_table_name)
2021-02-08 23:23:51 +00:00
: IStorage(table_id_)
2021-04-11 07:44:40 +00:00
, WithContext(context_->getGlobalContext())
2021-08-27 06:30:21 +00:00
, log(&Poco::Logger::get("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(postgres_database_name, postgres_table_name) + ")"))
2021-06-27 19:09:17 +00:00
, is_materialized_postgresql_database(true)
2021-05-03 09:52:13 +00:00
, has_nested(false)
2021-04-11 09:54:10 +00:00
, nested_context(makeNestedTableContext(context_->getGlobalContext()))
2021-08-27 06:30:21 +00:00
, nested_table_id(table_id_)
{
}
2021-06-27 19:09:17 +00:00
/// Constructor for MaterializedPostgreSQL table engine - for the case of MaterializePosgreSQL database engine.
2021-05-03 09:52:13 +00:00
/// It is used when nested ReplacingMergeeTree table has already been created by replication thread.
/// This storage is ready to handle read queries.
2021-08-27 06:30:21 +00:00
StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(
StoragePtr nested_storage_,
ContextPtr context_,
const String & postgres_database_name,
const String & postgres_table_name)
2021-11-05 14:25:02 +00:00
: IStorage(StorageID(nested_storage_->getStorageID().database_name, nested_storage_->getStorageID().table_name))
2021-05-02 11:50:29 +00:00
, WithContext(context_->getGlobalContext())
2021-08-27 06:30:21 +00:00
, log(&Poco::Logger::get("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(postgres_database_name, postgres_table_name) + ")"))
2021-06-27 19:09:17 +00:00
, is_materialized_postgresql_database(true)
2021-05-03 09:52:13 +00:00
, has_nested(true)
2021-05-02 11:50:29 +00:00
, nested_context(makeNestedTableContext(context_->getGlobalContext()))
2021-05-03 09:52:13 +00:00
, nested_table_id(nested_storage_->getStorageID())
2021-05-02 11:50:29 +00:00
{
setInMemoryMetadata(nested_storage_->getInMemoryMetadata());
}
2021-05-01 11:49:45 +00:00
/// A temporary clone table might be created for current table in order to update its schema and reload
/// all data in the background while current table will still handle read requests.
2021-06-27 19:09:17 +00:00
StoragePtr StorageMaterializedPostgreSQL::createTemporary() const
2021-05-01 11:49:45 +00:00
{
auto table_id = getStorageID();
2021-06-27 19:09:17 +00:00
auto tmp_table_id = StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX);
2021-05-03 09:52:13 +00:00
2021-06-27 19:09:17 +00:00
/// If for some reason it already exists - drop it.
auto tmp_storage = DatabaseCatalog::instance().tryGetTable(tmp_table_id, nested_context);
if (tmp_storage)
{
LOG_TRACE(&Poco::Logger::get("MaterializedPostgreSQLStorage"), "Temporary table {} already exists, dropping", tmp_table_id.getNameForLogs());
2023-05-03 18:06:46 +00:00
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), getContext(), tmp_table_id, /* sync */true);
2021-06-27 19:09:17 +00:00
}
auto new_context = Context::createCopy(context);
return std::make_shared<StorageMaterializedPostgreSQL>(tmp_table_id, new_context, "temporary", table_id.table_name);
2021-05-01 11:49:45 +00:00
}
2021-06-27 19:09:17 +00:00
StoragePtr StorageMaterializedPostgreSQL::getNested() const
{
2021-05-03 17:28:54 +00:00
return DatabaseCatalog::instance().getTable(getNestedStorageID(), nested_context);
}
2021-06-27 19:09:17 +00:00
StoragePtr StorageMaterializedPostgreSQL::tryGetNested() const
2021-02-08 23:23:51 +00:00
{
2021-05-03 17:28:54 +00:00
return DatabaseCatalog::instance().tryGetTable(getNestedStorageID(), nested_context);
2021-02-08 23:23:51 +00:00
}
2021-06-27 19:09:17 +00:00
String StorageMaterializedPostgreSQL::getNestedTableName() const
2021-02-08 23:23:51 +00:00
{
auto table_id = getStorageID();
2021-02-08 23:23:51 +00:00
2021-06-27 19:09:17 +00:00
if (is_materialized_postgresql_database)
return table_id.table_name;
2021-02-08 23:23:51 +00:00
return toString(table_id.uuid) + NESTED_TABLE_SUFFIX;
2021-02-08 23:23:51 +00:00
}
2021-06-27 19:09:17 +00:00
StorageID StorageMaterializedPostgreSQL::getNestedStorageID() const
2021-05-03 17:28:54 +00:00
{
if (nested_table_id.has_value())
return nested_table_id.value();
auto table_id = getStorageID();
throw Exception(ErrorCodes::LOGICAL_ERROR,
2021-06-27 19:09:17 +00:00
"No storageID found for inner table. ({})", table_id.getNameForLogs());
2021-05-03 17:28:54 +00:00
}
2021-12-14 13:53:47 +00:00
void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override)
2021-05-01 11:49:45 +00:00
{
2021-08-27 12:50:45 +00:00
if (tryGetNested())
return;
2021-05-01 11:49:45 +00:00
try
{
2021-12-14 13:53:47 +00:00
const auto ast_create = getCreateNestedTableQuery(std::move(table_structure), table_override);
auto table_id = getStorageID();
auto tmp_nested_table_id = StorageID(table_id.database_name, getNestedTableName());
LOG_DEBUG(log, "Creating clickhouse table for postgresql table {}", table_id.getNameForLogs());
2021-05-01 11:49:45 +00:00
InterpreterCreateQuery interpreter(ast_create, nested_context);
interpreter.execute();
2021-05-03 17:28:54 +00:00
2021-06-27 19:09:17 +00:00
auto nested_storage = DatabaseCatalog::instance().getTable(tmp_nested_table_id, nested_context);
2021-05-03 17:28:54 +00:00
/// Save storage_id with correct uuid.
nested_table_id = nested_storage->getStorageID();
2021-05-01 11:49:45 +00:00
}
2021-12-14 13:53:47 +00:00
catch (...)
2021-05-01 11:49:45 +00:00
{
tryLogCurrentException(__PRETTY_FUNCTION__);
2021-12-14 13:53:47 +00:00
throw;
2021-05-01 11:49:45 +00:00
}
}
2021-06-27 19:09:17 +00:00
std::shared_ptr<Context> StorageMaterializedPostgreSQL::makeNestedTableContext(ContextPtr from_context)
2021-05-01 11:49:45 +00:00
{
auto new_context = Context::createCopy(from_context);
new_context->setInternalQuery(true);
2021-05-01 11:49:45 +00:00
return new_context;
}
2021-11-05 14:25:02 +00:00
void StorageMaterializedPostgreSQL::set(StoragePtr nested_storage)
2021-05-03 09:52:13 +00:00
{
2021-11-05 14:25:02 +00:00
nested_table_id = nested_storage->getStorageID();
setInMemoryMetadata(nested_storage->getInMemoryMetadata());
2021-05-03 09:52:13 +00:00
has_nested.store(true);
}
2021-06-27 19:09:17 +00:00
void StorageMaterializedPostgreSQL::shutdown()
2021-05-01 11:49:45 +00:00
{
if (replication_handler)
replication_handler->shutdown();
2021-07-05 03:53:48 +00:00
auto nested = tryGetNested();
2021-07-01 07:33:58 +00:00
if (nested)
nested->shutdown();
2021-05-01 11:49:45 +00:00
}
2022-06-23 07:59:13 +00:00
void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool sync, ContextPtr local_context)
2021-05-01 11:49:45 +00:00
{
2021-07-01 08:20:13 +00:00
/// If it is a table with database engine MaterializedPostgreSQL - return, because delition of
2021-06-27 19:09:17 +00:00
/// internal tables is managed there.
if (is_materialized_postgresql_database)
return;
replication_handler->shutdownFinal();
2022-04-06 12:35:19 +00:00
replication_handler.reset();
2021-05-01 11:49:45 +00:00
2022-04-06 12:35:19 +00:00
auto nested_table = tryGetNested() != nullptr;
2021-06-27 19:09:17 +00:00
if (nested_table)
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), sync, /* ignore_sync_setting */ true);
2021-05-01 11:49:45 +00:00
}
2021-06-27 19:09:17 +00:00
NamesAndTypesList StorageMaterializedPostgreSQL::getVirtuals() const
2021-05-01 11:49:45 +00:00
{
return NamesAndTypesList{
{"_sign", std::make_shared<DataTypeInt8>()},
{"_version", std::make_shared<DataTypeUInt64>()}
};
}
bool StorageMaterializedPostgreSQL::needRewriteQueryWithFinal(const Names & column_names) const
{
return needRewriteQueryWithFinalForStorage(column_names, getNested());
}
2022-05-20 19:49:31 +00:00
void StorageMaterializedPostgreSQL::read(
QueryPlan & query_plan,
2021-05-01 11:49:45 +00:00
const Names & column_names,
const StorageSnapshotPtr & /*storage_snapshot*/,
2021-05-01 11:49:45 +00:00
SelectQueryInfo & query_info,
ContextPtr context_,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams)
2021-05-01 11:49:45 +00:00
{
auto nested_table = getNested();
2022-02-08 15:41:55 +00:00
2022-05-20 19:49:31 +00:00
readFinalFromNestedStorage(query_plan, nested_table, column_names,
2021-05-03 09:52:13 +00:00
query_info, context_, processed_stage, max_block_size, num_streams);
2022-02-08 15:41:55 +00:00
auto lock = lockForShare(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout);
2022-05-20 19:49:31 +00:00
query_plan.addTableLock(lock);
query_plan.addStorageHolder(shared_from_this());
2021-05-01 11:49:45 +00:00
}
2021-06-27 19:09:17 +00:00
std::shared_ptr<ASTColumnDeclaration> StorageMaterializedPostgreSQL::getMaterializedColumnsDeclaration(
String name, String type, UInt64 default_value)
{
auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = std::move(name);
column_declaration->type = makeASTFunction(type);
column_declaration->default_specifier = "MATERIALIZED";
column_declaration->default_expression = std::make_shared<ASTLiteral>(default_value);
column_declaration->children.emplace_back(column_declaration->type);
column_declaration->children.emplace_back(column_declaration->default_expression);
return column_declaration;
}
2021-06-27 19:09:17 +00:00
ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) const
{
WhichDataType which(data_type);
if (which.isNullable())
return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast<const DataTypeNullable *>(data_type.get())->getNestedType()));
if (which.isArray())
return makeASTFunction("Array", getColumnDeclaration(typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType()));
/// getName() for decimal returns 'Decimal(precision, scale)', will get an error with it
2021-02-12 10:05:13 +00:00
if (which.isDecimal())
{
auto make_decimal_expression = [&](std::string type_name)
{
auto ast_expression = std::make_shared<ASTFunction>();
ast_expression->name = type_name;
ast_expression->arguments = std::make_shared<ASTExpressionList>();
ast_expression->arguments->children.emplace_back(std::make_shared<ASTLiteral>(getDecimalScale(*data_type)));
return ast_expression;
};
if (which.isDecimal32())
return make_decimal_expression("Decimal32");
if (which.isDecimal64())
return make_decimal_expression("Decimal64");
if (which.isDecimal128())
return make_decimal_expression("Decimal128");
if (which.isDecimal256())
return make_decimal_expression("Decimal256");
}
if (which.isDateTime64())
{
auto ast_expression = std::make_shared<ASTFunction>();
ast_expression->name = "DateTime64";
ast_expression->arguments = std::make_shared<ASTExpressionList>();
ast_expression->arguments->children.emplace_back(std::make_shared<ASTLiteral>(static_cast<UInt32>(6)));
return ast_expression;
}
2021-02-12 10:05:13 +00:00
return std::make_shared<ASTIdentifier>(data_type->getName());
}
2021-12-14 13:53:47 +00:00
std::shared_ptr<ASTExpressionList> StorageMaterializedPostgreSQL::getColumnsExpressionList(const NamesAndTypesList & columns) const
{
auto columns_expression_list = std::make_shared<ASTExpressionList>();
for (const auto & [name, type] : columns)
{
const auto & column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = name;
column_declaration->type = getColumnDeclaration(type);
columns_expression_list->children.emplace_back(column_declaration);
}
return columns_expression_list;
}
2021-06-27 19:09:17 +00:00
/// For single storage MaterializedPostgreSQL get columns and primary key columns from storage definition.
/// For database engine MaterializedPostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same
2021-02-11 21:59:58 +00:00
/// transaction with snapshot, which is used for initial tables dump.
2021-12-14 13:53:47 +00:00
ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(
PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override)
{
2021-02-11 21:59:58 +00:00
auto create_table_query = std::make_shared<ASTCreateQuery>();
auto table_id = getStorageID();
create_table_query->setTable(getNestedTableName());
create_table_query->setDatabase(table_id.database_name);
2021-06-27 19:09:17 +00:00
if (is_materialized_postgresql_database)
2021-05-02 13:49:28 +00:00
create_table_query->uuid = table_id.uuid;
2021-12-14 13:53:47 +00:00
auto storage = std::make_shared<ASTStorage>();
storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared<ASTIdentifier>("_version")));
2021-02-11 21:59:58 +00:00
auto columns_declare_list = std::make_shared<ASTColumns>();
auto order_by_expression = std::make_shared<ASTFunction>();
auto metadata_snapshot = getInMemoryMetadataPtr();
2021-12-14 13:53:47 +00:00
ConstraintsDescription constraints;
2021-02-11 21:59:58 +00:00
NamesAndTypesList ordinary_columns_and_types;
2021-12-14 13:53:47 +00:00
if (is_materialized_postgresql_database)
2021-02-11 21:59:58 +00:00
{
2021-12-14 13:53:47 +00:00
if (!table_structure && !table_override)
2021-03-20 13:13:30 +00:00
{
2021-12-14 13:53:47 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "No table structure returned for table {}.{}",
table_id.database_name, table_id.table_name);
2021-03-20 13:13:30 +00:00
}
2021-02-11 21:59:58 +00:00
2022-01-08 17:44:17 +00:00
if (!table_structure->physical_columns && (!table_override || !table_override->columns))
2021-02-11 21:59:58 +00:00
{
2021-12-14 13:53:47 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns returned for table {}.{}",
table_id.database_name, table_id.table_name);
2021-02-11 21:59:58 +00:00
}
2021-02-08 23:23:51 +00:00
2021-12-14 13:53:47 +00:00
bool has_order_by_override = table_override && table_override->storage && table_override->storage->order_by;
if (has_order_by_override && !table_structure->replica_identity_columns)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Having PRIMARY KEY OVERRIDE is allowed only if there is "
"replica identity index for PostgreSQL table. (table {}.{})",
table_id.database_name, table_id.table_name);
}
2021-02-11 21:59:58 +00:00
2021-04-10 14:42:45 +00:00
if (!table_structure->primary_key_columns && !table_structure->replica_identity_columns)
2021-02-11 21:59:58 +00:00
{
2021-04-10 14:42:45 +00:00
throw Exception(ErrorCodes::BAD_ARGUMENTS,
2021-12-14 13:53:47 +00:00
"Table {}.{} has no primary key and no replica identity index",
table_id.database_name, table_id.table_name);
2021-02-11 21:59:58 +00:00
}
2021-12-14 13:53:47 +00:00
if (table_override && table_override->columns)
{
if (table_override->columns)
{
auto children = table_override->columns->children;
const auto & columns = children[0]->as<ASTExpressionList>();
if (columns)
{
for (const auto & child : columns->children)
{
const auto * column_declaration = child->as<ASTColumnDeclaration>();
auto type = DataTypeFactory::instance().get(column_declaration->type);
ordinary_columns_and_types.emplace_back(NameAndTypePair(column_declaration->name, type));
}
}
columns_declare_list->set(columns_declare_list->columns, children[0]);
}
else
{
2022-01-08 17:44:17 +00:00
ordinary_columns_and_types = table_structure->physical_columns->columns;
2021-12-14 13:53:47 +00:00
columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types));
}
2021-12-20 22:19:51 +00:00
auto * columns = table_override->columns;
2021-12-14 13:53:47 +00:00
if (columns && columns->constraints)
constraints = ConstraintsDescription(columns->constraints->children);
}
else
{
2022-01-08 17:44:17 +00:00
ordinary_columns_and_types = table_structure->physical_columns->columns;
2021-12-14 13:53:47 +00:00
columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types));
}
if (ordinary_columns_and_types.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Table {}.{} has no columns", table_id.database_name, table_id.table_name);
2021-04-10 14:42:45 +00:00
NamesAndTypesList merging_columns;
if (table_structure->primary_key_columns)
2022-01-08 17:44:17 +00:00
merging_columns = table_structure->primary_key_columns->columns;
2021-04-10 14:42:45 +00:00
else
2022-01-08 17:44:17 +00:00
merging_columns = table_structure->replica_identity_columns->columns;
2021-02-11 21:59:58 +00:00
order_by_expression->name = "tuple";
order_by_expression->arguments = std::make_shared<ASTExpressionList>();
2021-04-10 14:42:45 +00:00
for (const auto & column : merging_columns)
2021-02-11 21:59:58 +00:00
order_by_expression->arguments->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
2021-12-14 13:53:47 +00:00
storage->set(storage->order_by, order_by_expression);
}
else
{
2021-12-14 13:53:47 +00:00
ordinary_columns_and_types = metadata_snapshot->getColumns().getOrdinary();
columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types));
2021-02-11 21:59:58 +00:00
2021-12-14 13:53:47 +00:00
auto primary_key_ast = metadata_snapshot->getPrimaryKeyAST();
if (!primary_key_ast)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializedPostgreSQL must have primary key");
storage->set(storage->order_by, primary_key_ast);
2021-02-11 21:59:58 +00:00
2021-12-14 13:53:47 +00:00
constraints = metadata_snapshot->getConstraints();
}
2021-02-08 23:23:51 +00:00
create_table_query->set(create_table_query->storage, storage);
2022-03-09 18:25:43 +00:00
if (table_override)
2022-03-10 11:07:16 +00:00
{
if (auto * columns = table_override->columns)
{
if (columns->columns)
{
for (const auto & override_column_ast : columns->columns->children)
{
auto * override_column = override_column_ast->as<ASTColumnDeclaration>();
if (override_column->name == "_sign" || override_column->name == "_version")
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot override _sign and _version column");
}
}
}
create_table_query->set(create_table_query->columns_list, columns_declare_list);
2022-03-09 18:25:43 +00:00
applyTableOverrideToCreateQuery(*table_override, create_table_query.get());
2022-03-10 11:07:16 +00:00
create_table_query->columns_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", 1));
create_table_query->columns_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_version", "UInt64", 1));
}
else
{
columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", 1));
columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_version", "UInt64", 1));
create_table_query->set(create_table_query->columns_list, columns_declare_list);
}
/// Add columns _sign and _version, so that they can be accessed from nested ReplacingMergeTree table if needed.
ordinary_columns_and_types.push_back({"_sign", std::make_shared<DataTypeInt8>()});
ordinary_columns_and_types.push_back({"_version", std::make_shared<DataTypeUInt64>()});
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types));
2021-12-14 13:53:47 +00:00
storage_metadata.setConstraints(constraints);
setInMemoryMetadata(storage_metadata);
return create_table_query;
}
2021-06-27 19:09:17 +00:00
void registerStorageMaterializedPostgreSQL(StorageFactory & factory)
{
auto creator_fn = [](const StorageFactory::Arguments & args)
{
StorageInMemoryMetadata metadata;
metadata.setColumns(args.columns);
metadata.setConstraints(args.constraints);
if (!args.storage_def->order_by && args.storage_def->primary_key)
args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone());
if (!args.storage_def->order_by)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializedPostgreSQL needs order by key or primary key");
if (args.storage_def->primary_key)
2021-04-11 07:44:40 +00:00
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext());
else
2021-04-11 07:44:40 +00:00
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.getContext());
2021-09-03 11:16:32 +00:00
auto configuration = StoragePostgreSQL::getConfiguration(args.engine_args, args.getContext());
2021-04-09 10:22:06 +00:00
auto connection_info = postgres::formatConnectionString(
2021-09-03 11:16:32 +00:00
configuration.database, configuration.host, configuration.port,
configuration.username, configuration.password);
bool has_settings = args.storage_def->settings;
auto postgresql_replication_settings = std::make_unique<MaterializedPostgreSQLSettings>();
if (has_settings)
postgresql_replication_settings->loadFromQuery(*args.storage_def);
return std::make_shared<StorageMaterializedPostgreSQL>(
2021-09-03 11:16:32 +00:00
args.table_id, args.attach, configuration.database, configuration.table, connection_info,
2021-04-11 07:44:40 +00:00
metadata, args.getContext(),
std::move(postgresql_replication_settings));
};
factory.registerStorage(
2021-06-27 19:09:17 +00:00
"MaterializedPostgreSQL",
creator_fn,
2021-02-11 21:59:58 +00:00
StorageFactory::StorageFeatures{
.supports_settings = true,
.supports_sort_order = true,
.source_access_type = AccessType::POSTGRES,
});
}
}
2021-02-18 06:06:37 +00:00
#endif