2020-11-20 22:47:04 +00:00
|
|
|
#include "StoragePostgreSQL.h"
|
|
|
|
|
|
|
|
#if USE_LIBPQXX
|
|
|
|
|
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Storages/transformQueryForExternalDatabase.h>
|
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <DataTypes/DataTypeString.h>
|
2021-01-09 22:42:42 +00:00
|
|
|
#include <DataTypes/DataTypeArray.h>
|
|
|
|
#include <DataTypes/DataTypeDate.h>
|
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
|
|
|
#include <DataTypes/DataTypesDecimal.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#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 <DataStreams/PostgreSQLBlockInputStream.h>
|
2020-11-20 22:47:04 +00:00
|
|
|
#include <Core/Settings.h>
|
|
|
|
#include <Common/parseAddress.h>
|
2020-11-25 14:52:11 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2020-11-20 22:47:04 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
2020-11-25 14:52:11 +00:00
|
|
|
#include <Columns/ColumnNullable.h>
|
2020-11-20 22:47:04 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2020-11-25 14:52:11 +00:00
|
|
|
#include <Formats/FormatSettings.h>
|
2020-11-20 22:47:04 +00:00
|
|
|
#include <Processors/Sources/SourceFromInputStream.h>
|
|
|
|
#include <Processors/Pipe.h>
|
2020-11-25 21:06:33 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2020-11-20 22:47:04 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
|
2020-11-20 22:47:04 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2021-01-09 22:42:42 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2020-11-20 22:47:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
StoragePostgreSQL::StoragePostgreSQL(
|
|
|
|
const StorageID & table_id_,
|
|
|
|
const String & remote_table_name_,
|
2021-01-09 22:42:42 +00:00
|
|
|
PostgreSQLConnectionPtr connection_,
|
2020-11-20 22:47:04 +00:00
|
|
|
const ColumnsDescription & columns_,
|
|
|
|
const ConstraintsDescription & constraints_,
|
|
|
|
const Context & context_)
|
|
|
|
: IStorage(table_id_)
|
|
|
|
, remote_table_name(remote_table_name_)
|
|
|
|
, global_context(context_)
|
2020-11-25 21:06:33 +00:00
|
|
|
, connection(std::move(connection_))
|
2020-11-20 22:47:04 +00:00
|
|
|
{
|
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
storage_metadata.setConstraints(constraints_);
|
|
|
|
setInMemoryMetadata(storage_metadata);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
Pipe StoragePostgreSQL::read(
|
|
|
|
const Names & column_names_,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
|
|
|
SelectQueryInfo & query_info_,
|
|
|
|
const Context & context_,
|
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t max_block_size_,
|
|
|
|
unsigned)
|
|
|
|
{
|
|
|
|
metadata_snapshot->check(column_names_, getVirtuals(), getStorageID());
|
|
|
|
|
|
|
|
String query = transformQueryForExternalDatabase(
|
2020-11-25 21:06:33 +00:00
|
|
|
query_info_, metadata_snapshot->getColumns().getOrdinary(),
|
|
|
|
IdentifierQuotingStyle::DoubleQuotes, "", remote_table_name, context_);
|
2020-11-20 22:47:04 +00:00
|
|
|
|
|
|
|
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<SourceFromInputStream>(
|
2020-12-21 19:20:56 +00:00
|
|
|
std::make_shared<PostgreSQLBlockInputStream>(connection->conn(), query, sample_block, max_block_size_)));
|
2020-11-20 22:47:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
class PostgreSQLBlockOutputStream : public IBlockOutputStream
|
2020-11-25 14:52:11 +00:00
|
|
|
{
|
2021-01-09 22:42:42 +00:00
|
|
|
public:
|
|
|
|
explicit PostgreSQLBlockOutputStream(
|
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
|
|
|
ConnectionPtr connection_,
|
|
|
|
const std::string & remote_table_name_)
|
|
|
|
: metadata_snapshot(metadata_snapshot_)
|
|
|
|
, connection(connection_)
|
|
|
|
, remote_table_name(remote_table_name_)
|
|
|
|
{
|
|
|
|
}
|
2020-11-25 14:52:11 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
Block getHeader() const override { return metadata_snapshot->getSampleBlock(); }
|
2020-11-25 14:52:11 +00:00
|
|
|
|
2021-01-10 10:03:01 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
void writePrefix() override
|
|
|
|
{
|
|
|
|
work = std::make_unique<pqxx::work>(*connection);
|
|
|
|
}
|
2020-11-25 14:52:11 +00:00
|
|
|
|
2021-01-10 10:03:01 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
void write(const Block & block) override
|
|
|
|
{
|
|
|
|
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
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
if (!stream_inserter)
|
|
|
|
stream_inserter = std::make_unique<pqxx::stream_to>(*work, remote_table_name, block.getNames());
|
2020-12-08 21:40:18 +00:00
|
|
|
|
2021-01-09 22:42:42 +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-01-09 22:42:42 +00:00
|
|
|
for (const auto i : ext::range(0, num_rows))
|
|
|
|
{
|
|
|
|
for (const auto j : ext::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
|
|
|
|
{
|
|
|
|
data_types[j]->serializeAsText(*columns[j], i, ostr, FormatSettings{});
|
|
|
|
}
|
|
|
|
|
|
|
|
row[j] = ostr.str();
|
|
|
|
}
|
|
|
|
}
|
2021-01-10 10:03:01 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
stream_inserter->write_values(row);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-10 10:03:01 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
void writeSuffix() override
|
|
|
|
{
|
|
|
|
if (stream_inserter)
|
2021-01-10 10:03:01 +00:00
|
|
|
{
|
2021-01-09 22:42:42 +00:00
|
|
|
stream_inserter->complete();
|
2021-01-10 10:03:01 +00:00
|
|
|
work->commit();
|
|
|
|
}
|
2021-01-09 22:42:42 +00:00
|
|
|
}
|
2020-11-25 14:52:11 +00:00
|
|
|
|
2021-01-10 10:03:01 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
/// Cannot just use serializeAsText for array data type even though it converts perfectly
|
|
|
|
/// any dimension number array into text format, because it incloses in '[]' and for postgres it must be '{}'.
|
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
|
|
|
{
|
2021-01-09 22:42:42 +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();
|
2021-01-09 22:42:42 +00:00
|
|
|
const auto & array = array_field.get<Array>();
|
|
|
|
|
|
|
|
if (!isArray(nested))
|
2020-11-25 14:52:11 +00:00
|
|
|
{
|
2021-01-09 22:42:42 +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
|
|
|
{
|
2021-01-09 22:42:42 +00:00
|
|
|
writeText(clickhouseToPostgresArray(iter->get<Array>(), nested), ostr);
|
2020-11-25 14:52:11 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-01-09 22:42:42 +00:00
|
|
|
parseArray(*iter, nested, ostr);
|
2020-11-25 14:52:11 +00:00
|
|
|
}
|
|
|
|
}
|
2021-01-09 22:42:42 +00:00
|
|
|
|
|
|
|
writeChar('}', ostr);
|
2020-11-25 14:52:11 +00:00
|
|
|
}
|
|
|
|
|
2021-01-10 10:03:01 +00:00
|
|
|
|
2021-01-09 22:42:42 +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 10:03:01 +00:00
|
|
|
std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type)
|
2021-01-09 22:42:42 +00:00
|
|
|
{
|
|
|
|
auto nested = typeid_cast<const DataTypeArray *>(data_type.get())->getNestedType();
|
|
|
|
ColumnPtr nested_column(createNested(nested));
|
2021-01-10 10:03:01 +00:00
|
|
|
auto array_column{ColumnArray::create(nested_column)};
|
2021-01-09 22:42:42 +00:00
|
|
|
const_cast<ColumnArray *>(array_column.get())->insert(array_field);
|
|
|
|
WriteBufferFromOwnString ostr;
|
|
|
|
data_type->serializeAsText(*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() == '[]'
|
|
|
|
return '{' + std::string(ostr.str().begin() + 1, ostr.str().end() - 1) + '}';
|
|
|
|
}
|
|
|
|
|
2021-01-10 10:03:01 +00:00
|
|
|
|
|
|
|
static ColumnPtr createNested(DataTypePtr nested)
|
2021-01-09 22:42:42 +00:00
|
|
|
{
|
|
|
|
bool is_nullable = false;
|
|
|
|
if (nested->isNullable())
|
|
|
|
{
|
|
|
|
is_nullable = true;
|
|
|
|
nested = typeid_cast<const DataTypeNullable *>(nested.get())->getNestedType();
|
|
|
|
}
|
|
|
|
|
|
|
|
WhichDataType which(nested);
|
|
|
|
ColumnPtr 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.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());
|
|
|
|
}
|
2021-01-09 22:42:42 +00:00
|
|
|
else
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversion not supported");
|
|
|
|
|
|
|
|
if (is_nullable)
|
|
|
|
return ColumnNullable::create(nested_column, ColumnUInt8::create(nested_column->size(), 0));
|
|
|
|
|
|
|
|
return nested_column;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
private:
|
|
|
|
StorageMetadataPtr metadata_snapshot;
|
|
|
|
ConnectionPtr connection;
|
|
|
|
std::string remote_table_name;
|
|
|
|
|
|
|
|
std::unique_ptr<pqxx::work> work;
|
|
|
|
std::unique_ptr<pqxx::stream_to> stream_inserter;
|
|
|
|
};
|
2020-11-25 14:52:11 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
|
|
|
|
BlockOutputStreamPtr StoragePostgreSQL::write(
|
|
|
|
const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */)
|
2020-11-25 14:52:11 +00:00
|
|
|
{
|
2021-01-09 22:42:42 +00:00
|
|
|
return std::make_shared<PostgreSQLBlockOutputStream>(metadata_snapshot, connection->conn(), remote_table_name);
|
2020-11-25 14:52:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-11-20 22:47:04 +00:00
|
|
|
void registerStoragePostgreSQL(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
|
2020-12-08 21:40:18 +00:00
|
|
|
if (engine_args.size() != 5)
|
|
|
|
throw Exception("Storage PostgreSQL requires 5 parameters: "
|
|
|
|
"PostgreSQL('host:port', 'database', 'table', 'username', 'password'.",
|
2020-11-20 22:47:04 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
for (auto & engine_arg : engine_args)
|
|
|
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context);
|
|
|
|
|
|
|
|
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 5432);
|
|
|
|
const String & remote_table = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
|
|
|
|
String connection_str;
|
2020-11-25 21:06:33 +00:00
|
|
|
connection_str = fmt::format("dbname={} host={} port={} user={} password={}",
|
|
|
|
engine_args[1]->as<ASTLiteral &>().value.safeGet<String>(),
|
|
|
|
parsed_host_port.first, std::to_string(parsed_host_port.second),
|
|
|
|
engine_args[3]->as<ASTLiteral &>().value.safeGet<String>(),
|
|
|
|
engine_args[4]->as<ASTLiteral &>().value.safeGet<String>());
|
2020-11-20 22:47:04 +00:00
|
|
|
|
2021-01-09 22:42:42 +00:00
|
|
|
auto connection = std::make_shared<PostgreSQLConnection>(connection_str);
|
2020-11-20 22:47:04 +00:00
|
|
|
return StoragePostgreSQL::create(
|
2020-12-21 19:20:56 +00:00
|
|
|
args.table_id, remote_table, connection, args.columns, args.constraints, args.context);
|
2020-11-20 22:47:04 +00:00
|
|
|
},
|
|
|
|
{
|
|
|
|
.source_access_type = AccessType::POSTGRES,
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
#endif
|