ClickHouse/src/Storages/StoragePostgreSQL.cpp

461 lines
17 KiB
C++
Raw Normal View History

#include "StoragePostgreSQL.h"
#if USE_LIBPQXX
2021-08-11 16:44:34 +00:00
#include <DataStreams/PostgreSQLSource.h>
2021-09-01 17:59:11 +00:00
#include <Common/parseAddress.h>
#include <Common/assert_cast.h>
#include <Common/parseRemoteDescription.h>
#include <Core/Settings.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypeNullable.h>
2021-09-01 17:59:11 +00:00
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnDecimal.h>
2020-11-25 14:52:11 +00:00
#include <Columns/ColumnNullable.h>
2021-09-01 17:59:11 +00:00
#include <Formats/FormatFactory.h>
2020-11-25 14:52:11 +00:00
#include <Formats/FormatSettings.h>
2021-09-01 17:59:11 +00:00
2020-11-25 21:06:33 +00:00
#include <IO/WriteHelpers.h>
2021-08-24 12:29:42 +00:00
#include <IO/Operators.h>
2021-09-01 17:59:11 +00:00
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
#include <Parsers/getInsertQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <Storages/StorageFactory.h>
#include <Storages/transformQueryForExternalDatabase.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NOT_IMPLEMENTED;
}
StoragePostgreSQL::StoragePostgreSQL(
const StorageID & table_id_,
2021-05-07 11:18:49 +00:00
postgres::PoolWithFailoverPtr pool_,
const String & remote_table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
2021-04-23 12:18:23 +00:00
const String & comment,
2021-08-24 12:29:42 +00:00
const String & remote_table_schema_,
const String & on_conflict_)
: IStorage(table_id_)
, remote_table_name(remote_table_name_)
, remote_table_schema(remote_table_schema_)
2021-08-24 12:29:42 +00:00
, on_conflict(on_conflict_)
2021-05-07 11:18:49 +00:00
, pool(std::move(pool_))
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
2021-04-23 12:18:23 +00:00
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
}
Pipe StoragePostgreSQL::read(
const Names & column_names_,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & query_info_,
ContextPtr context_,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size_,
unsigned)
{
metadata_snapshot->check(column_names_, getVirtuals(), getStorageID());
/// Connection is already made to the needed database, so it should not be present in the query;
/// remote_table_schema is empty if it is not specified, will access only table_name.
String query = transformQueryForExternalDatabase(
2020-11-25 21:06:33 +00:00
query_info_, metadata_snapshot->getColumns().getOrdinary(),
IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_);
Block sample_block;
for (const String & column_name : column_names_)
{
auto column_data = metadata_snapshot->getColumns().getPhysical(column_name);
WhichDataType which(column_data.type);
if (which.isEnum())
column_data.type = std::make_shared<DataTypeString>();
sample_block.insert({ column_data.type, column_data.name });
}
return Pipe(std::make_shared<PostgreSQLSource<>>(pool->get(), query, sample_block, max_block_size_));
}
2021-07-23 19:33:59 +00:00
class PostgreSQLSink : public SinkToStorage
2020-11-25 14:52:11 +00:00
{
2021-08-24 12:29:42 +00:00
using Row = std::vector<std::optional<std::string>>;
public:
2021-07-23 19:33:59 +00:00
explicit PostgreSQLSink(
const StorageMetadataPtr & metadata_snapshot_,
2021-05-07 11:18:49 +00:00
postgres::ConnectionHolderPtr connection_holder_,
2021-05-22 08:47:19 +00:00
const String & remote_table_name_,
2021-08-24 12:29:42 +00:00
const String & remote_table_schema_,
const String & on_conflict_)
2021-07-26 10:08:40 +00:00
: SinkToStorage(metadata_snapshot_->getSampleBlock())
2021-07-23 19:33:59 +00:00
, metadata_snapshot(metadata_snapshot_)
2021-05-07 11:18:49 +00:00
, connection_holder(std::move(connection_holder_))
, remote_table_name(remote_table_name_)
2021-05-22 08:47:19 +00:00
, remote_table_schema(remote_table_schema_)
2021-08-24 12:29:42 +00:00
, on_conflict(on_conflict_)
{
}
2020-11-25 14:52:11 +00:00
2021-07-23 19:33:59 +00:00
String getName() const override { return "PostgreSQLSink"; }
2020-11-25 14:52:11 +00:00
2021-07-23 19:33:59 +00:00
void consume(Chunk chunk) override
{
2021-07-23 19:33:59 +00:00
auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns());
2021-08-24 12:29:42 +00:00
2021-05-22 08:47:19 +00:00
if (!inserter)
2021-08-24 12:29:42 +00:00
{
if (on_conflict.empty())
{
inserter = std::make_unique<StreamTo>(connection_holder->get(),
remote_table_schema.empty() ? pqxx::table_path({remote_table_name})
: pqxx::table_path({remote_table_schema, remote_table_name}), block.getNames());
}
else
{
inserter = std::make_unique<PreparedInsert>(connection_holder->get(), remote_table_name,
remote_table_schema, block.getColumnsWithTypeAndName(), on_conflict);
}
}
2021-01-10 13:01:15 +00:00
const auto columns = block.getColumns();
const size_t num_rows = block.rows(), num_cols = block.columns();
const auto data_types = block.getDataTypes();
2020-11-25 14:52:11 +00:00
/// std::optional lets libpqxx to know if value is NULL
std::vector<std::optional<std::string>> row(num_cols);
2020-11-25 14:52:11 +00:00
2021-06-15 19:55:21 +00:00
for (const auto i : collections::range(0, num_rows))
{
2021-06-15 19:55:21 +00:00
for (const auto j : collections::range(0, num_cols))
{
if (columns[j]->isNullAt(i))
{
row[j] = std::nullopt;
}
else
{
WriteBufferFromOwnString ostr;
if (isArray(data_types[j]))
{
parseArray((*columns[j])[i], data_types[j], ostr);
}
else
{
2021-03-09 14:46:52 +00:00
data_types[j]->getDefaultSerialization()->serializeText(*columns[j], i, ostr, FormatSettings{});
}
row[j] = ostr.str();
}
}
2021-01-10 10:03:01 +00:00
2021-08-24 12:29:42 +00:00
inserter->insert(row);
}
}
2021-07-23 19:33:59 +00:00
void onFinish() override
{
2021-05-22 08:47:19 +00:00
if (inserter)
inserter->complete();
}
2020-11-25 14:52:11 +00:00
/// Cannot just use serializeAsText for array data type even though it converts perfectly
2021-06-28 17:02:22 +00:00
/// any dimension number array into text format, because it encloses in '[]' and for postgres it must be '{}'.
2021-01-16 08:43:59 +00:00
/// Check if array[...] syntax from PostgreSQL will be applicable.
2021-01-10 10:03:01 +00:00
void parseArray(const Field & array_field, const DataTypePtr & data_type, WriteBuffer & ostr)
2020-11-25 14:52:11 +00:00
{
const auto * array_type = typeid_cast<const DataTypeArray *>(data_type.get());
2021-01-10 10:03:01 +00:00
const auto & nested = array_type->getNestedType();
const auto & array = array_field.get<Array>();
if (!isArray(nested))
2020-11-25 14:52:11 +00:00
{
writeText(clickhouseToPostgresArray(array, data_type), ostr);
return;
}
writeChar('{', ostr);
const auto * nested_array_type = typeid_cast<const DataTypeArray *>(nested.get());
for (auto iter = array.begin(); iter != array.end(); ++iter)
{
if (iter != array.begin())
writeText(", ", ostr);
if (!isArray(nested_array_type->getNestedType()))
2020-11-25 14:52:11 +00:00
{
writeText(clickhouseToPostgresArray(iter->get<Array>(), nested), ostr);
2020-11-25 14:52:11 +00:00
}
else
{
parseArray(*iter, nested, ostr);
2020-11-25 14:52:11 +00:00
}
}
writeChar('}', ostr);
2020-11-25 14:52:11 +00:00
}
/// Conversion is done via column casting because with writeText(Array..) got incorrect conversion
/// of Date and DateTime data types and it added extra quotes for values inside array.
2021-01-10 15:33:35 +00:00
static std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type)
{
auto nested = typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType();
2021-01-15 20:09:31 +00:00
auto array_column = ColumnArray::create(createNested(nested));
array_column->insert(array_field);
WriteBufferFromOwnString ostr;
2021-03-09 14:46:52 +00:00
data_type->getDefaultSerialization()->serializeText(*array_column, 0, ostr, FormatSettings{});
/// ostr is guaranteed to be at least '[]', i.e. size is at least 2 and 2 only if ostr.str() == '[]'
2021-01-16 08:30:35 +00:00
assert(ostr.str().size() >= 2);
return '{' + std::string(ostr.str().begin() + 1, ostr.str().end() - 1) + '}';
}
2021-01-15 20:09:31 +00:00
static MutableColumnPtr createNested(DataTypePtr nested)
{
bool is_nullable = false;
if (nested->isNullable())
{
is_nullable = true;
2021-01-10 15:33:35 +00:00
nested = static_cast<const DataTypeNullable *>(nested.get())->getNestedType();
}
WhichDataType which(nested);
2021-01-15 20:09:31 +00:00
MutableColumnPtr nested_column;
if (which.isString() || which.isFixedString()) nested_column = ColumnString::create();
else if (which.isInt8() || which.isInt16()) nested_column = ColumnInt16::create();
else if (which.isUInt8() || which.isUInt16()) nested_column = ColumnUInt16::create();
else if (which.isInt32()) nested_column = ColumnInt32::create();
else if (which.isUInt32()) nested_column = ColumnUInt32::create();
else if (which.isInt64()) nested_column = ColumnInt64::create();
else if (which.isUInt64()) nested_column = ColumnUInt64::create();
else if (which.isFloat32()) nested_column = ColumnFloat32::create();
else if (which.isFloat64()) nested_column = ColumnFloat64::create();
else if (which.isDate()) nested_column = ColumnUInt16::create();
else if (which.isDateTime()) nested_column = ColumnUInt32::create();
else if (which.isDateTime64())
{
nested_column = ColumnDecimal<DateTime64>::create(0, 6);
}
else if (which.isDecimal32())
{
const auto & type = typeid_cast<const DataTypeDecimal<Decimal32> *>(nested.get());
nested_column = ColumnDecimal<Decimal32>::create(0, type->getScale());
}
else if (which.isDecimal64())
{
const auto & type = typeid_cast<const DataTypeDecimal<Decimal64> *>(nested.get());
nested_column = ColumnDecimal<Decimal64>::create(0, type->getScale());
}
else if (which.isDecimal128())
{
const auto & type = typeid_cast<const DataTypeDecimal<Decimal128> *>(nested.get());
nested_column = ColumnDecimal<Decimal128>::create(0, type->getScale());
}
2021-01-10 10:03:01 +00:00
else if (which.isDecimal256())
{
const auto & type = typeid_cast<const DataTypeDecimal<Decimal256> *>(nested.get());
nested_column = ColumnDecimal<Decimal256>::create(0, type->getScale());
}
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversion not supported");
if (is_nullable)
2021-01-15 20:09:31 +00:00
return ColumnNullable::create(std::move(nested_column), ColumnUInt8::create(nested_column->size(), 0));
return nested_column;
}
private:
2021-08-24 12:29:42 +00:00
struct Inserter
2021-05-22 08:47:19 +00:00
{
2021-08-24 12:29:42 +00:00
pqxx::connection & connection;
2021-05-22 08:47:19 +00:00
pqxx::work tx;
2021-08-24 12:29:42 +00:00
2021-08-25 21:51:43 +00:00
explicit Inserter(pqxx::connection & connection_)
2021-08-24 12:29:42 +00:00
: connection(connection_)
, tx(connection) {}
virtual ~Inserter() = default;
virtual void insert(const Row & row) = 0;
virtual void complete() = 0;
};
struct StreamTo : Inserter
{
2021-06-09 07:46:49 +00:00
Names columns;
2021-05-22 08:47:19 +00:00
pqxx::stream_to stream;
2021-08-24 12:29:42 +00:00
StreamTo(pqxx::connection & connection_, pqxx::table_path table_, Names columns_)
: Inserter(connection_)
2021-06-09 07:46:49 +00:00
, columns(std::move(columns_))
2021-06-16 09:11:31 +00:00
, stream(pqxx::stream_to::raw_table(tx, connection.quote_table(table_), connection.quote_columns(columns)))
2021-06-09 07:46:49 +00:00
{
}
2021-05-22 08:47:19 +00:00
2021-08-24 12:29:42 +00:00
void complete() override
2021-05-22 08:47:19 +00:00
{
stream.complete();
tx.commit();
}
2021-08-24 12:29:42 +00:00
void insert(const Row & row) override
{
stream.write_values(row);
}
};
struct PreparedInsert : Inserter
{
PreparedInsert(pqxx::connection & connection_, const String & table, const String & schema,
const ColumnsWithTypeAndName & columns, const String & on_conflict_)
: Inserter(connection_)
{
WriteBufferFromOwnString buf;
buf << getInsertQuery(schema, table, columns, IdentifierQuotingStyle::DoubleQuotes);
buf << " (";
for (size_t i = 1; i <= columns.size(); ++i)
{
if (i > 1)
buf << ", ";
buf << "$" << i;
}
buf << ") ";
buf << on_conflict_;
connection.prepare("insert", buf.str());
}
void complete() override
{
connection.unprepare("insert");
tx.commit();
}
void insert(const Row & row) override
{
pqxx::params params;
params.reserve(row.size());
params.append_multi(row);
tx.exec_prepared("insert", params);
}
2021-05-22 08:47:19 +00:00
};
StorageMetadataPtr metadata_snapshot;
2021-05-07 11:18:49 +00:00
postgres::ConnectionHolderPtr connection_holder;
2021-08-24 12:29:42 +00:00
const String remote_db_name, remote_table_name, remote_table_schema, on_conflict;
std::unique_ptr<Inserter> inserter;
};
2020-11-25 14:52:11 +00:00
2021-07-23 19:33:59 +00:00
SinkToStoragePtr StoragePostgreSQL::write(
const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /* context */)
2020-11-25 14:52:11 +00:00
{
2021-08-24 12:29:42 +00:00
return std::make_shared<PostgreSQLSink>(metadata_snapshot, pool->get(), remote_table_name, remote_table_schema, on_conflict);
2020-11-25 14:52:11 +00:00
}
2021-09-01 17:59:11 +00:00
StoragePostgreSQLConfiguration StoragePostgreSQL::getConfiguration(ASTs engine_args, ContextPtr context)
{
2021-09-01 17:59:11 +00:00
auto [common_configuration, storage_specific_args, with_named_collection] = tryGetConfigurationAsNamedCollection(engine_args, context);
StoragePostgreSQLConfiguration configuration(common_configuration);
2021-09-01 17:59:11 +00:00
if (with_named_collection)
{
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
for (const auto & [arg_name, arg_value] : storage_specific_args)
{
2021-09-01 23:17:15 +00:00
if (arg_name == "on_conflict")
2021-09-01 17:59:11 +00:00
configuration.on_conflict = arg_value.safeGet<String>();
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unxpected argument name for key-value defined argument."
"Got: {}, but expected one of:"
"host, port, username, password, database, table, schema, on_conflict.", arg_name);
}
}
else
{
2021-08-24 12:29:42 +00:00
if (engine_args.size() < 5 || engine_args.size() > 7)
2021-09-01 17:59:11 +00:00
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage PostgreSQL requires from 5 to 7 parameters: "
"PostgreSQL('host:port', 'database', 'table', 'username', 'password' [, 'schema', 'ON CONFLICT ...']. Got: {}",
engine_args.size());
for (auto & engine_arg : engine_args)
2021-09-01 17:59:11 +00:00
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
2021-09-01 17:59:11 +00:00
const auto & host_port = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
2021-03-27 19:18:05 +00:00
2021-09-01 17:59:11 +00:00
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
configuration.table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
2021-08-24 12:29:42 +00:00
if (engine_args.size() >= 6)
2021-09-01 17:59:11 +00:00
configuration.schema = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
2021-08-24 12:29:42 +00:00
if (engine_args.size() >= 7)
2021-09-01 17:59:11 +00:00
configuration.on_conflict = engine_args[6]->as<ASTLiteral &>().value.safeGet<String>();
}
return configuration;
}
2021-09-01 17:59:11 +00:00
void registerStoragePostgreSQL(StorageFactory & factory)
{
factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args)
{
auto configuration = StoragePostgreSQL::getConfiguration(args.engine_args, args.getContext());
2021-09-01 23:17:15 +00:00
auto pool = std::make_shared<postgres::PoolWithFailover>(configuration,
args.getContext()->getSettingsRef().postgresql_connection_pool_size,
args.getContext()->getSettingsRef().postgresql_connection_pool_wait_timeout);
2021-01-15 21:18:51 +00:00
return StoragePostgreSQL::create(
2021-05-11 16:22:24 +00:00
args.table_id,
std::move(pool),
2021-09-01 17:59:11 +00:00
configuration.table,
2021-05-11 16:22:24 +00:00
args.columns,
args.constraints,
args.comment,
2021-09-01 17:59:11 +00:00
configuration.schema,
configuration.on_conflict);
},
{
.source_access_type = AccessType::POSTGRES,
});
}
}
#endif