ClickHouse/src/Storages/StoragePostgreSQL.h
Robert Schulze 330212e0f4
Remove inherited create() method + disallow copying
The original motivation for this commit was that shared_ptr_helper used
std::shared_ptr<>() which does two heap allocations instead of
make_shared<>() which does a single allocation. Turned out that
1. the affected code (--> Storages/) is not on a hot path (rendering the
performance argument moot ...)
2. yet copying Storage objects is potentially dangerous and was
   previously allowed.

Hence, this change

- removes shared_ptr_helper and as a result all inherited create() methods,

- instead, Storage objects are now created using make_shared<>() by the
  caller (for that to work, many constructors had to be made public), and

- all Storage classes were marked as noncopyable using boost::noncopyable.

In sum, we are (likely) not making things faster but the code becomes
cleaner and harder to misuse.
2022-05-02 08:46:52 +02:00

62 lines
1.5 KiB
C++

#pragma once
#include "config_core.h"
#if USE_LIBPQXX
#include <boost/noncopyable.hpp>
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
#include <Core/PostgreSQL/PoolWithFailover.h>
#include <Storages/ExternalDataSourceConfiguration.h>
namespace Poco
{
class Logger;
}
namespace DB
{
class StoragePostgreSQL final : public IStorage, boost::noncopyable
{
public:
StoragePostgreSQL(
const StorageID & table_id_,
postgres::PoolWithFailoverPtr pool_,
const String & remote_table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
const String & remote_table_schema_ = "",
const String & on_conflict = "");
String getName() const override { return "PostgreSQL"; }
Pipe read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
unsigned num_streams) override;
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
static StoragePostgreSQLConfiguration getConfiguration(ASTs engine_args, ContextPtr context);
private:
friend class PostgreSQLBlockOutputStream;
String remote_table_name;
String remote_table_schema;
String on_conflict;
postgres::PoolWithFailoverPtr pool;
Poco::Logger * log;
};
}
#endif