mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #20470 from kssenii/pg-ch-replica
MaterializePostgreSQL table engine and database engine
This commit is contained in:
commit
8351f1db99
@ -11,6 +11,7 @@ services:
|
||||
interval: 10s
|
||||
timeout: 5s
|
||||
retries: 5
|
||||
command: [ "postgres", "-c", "wal_level=logical", "-c", "max_replication_slots=2"]
|
||||
networks:
|
||||
default:
|
||||
aliases:
|
||||
|
71
docs/en/engines/database-engines/materialized-postgresql.md
Normal file
71
docs/en/engines/database-engines/materialized-postgresql.md
Normal file
@ -0,0 +1,71 @@
|
||||
---
|
||||
toc_priority: 30
|
||||
toc_title: MaterializedPostgreSQL
|
||||
---
|
||||
|
||||
# MaterializedPostgreSQL {#materialize-postgresql}
|
||||
|
||||
## Creating a Database {#creating-a-database}
|
||||
|
||||
``` sql
|
||||
CREATE DATABASE test_database
|
||||
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'
|
||||
|
||||
SELECT * FROM test_database.postgres_table;
|
||||
```
|
||||
|
||||
|
||||
## Settings {#settings}
|
||||
|
||||
1. `materialized_postgresql_max_block_size` - Number of rows collected before flushing data into table. Default: `65536`.
|
||||
|
||||
2. `materialized_postgresql_tables_list` - List of tables for MaterializedPostgreSQL database engine. Default: `whole database`.
|
||||
|
||||
3. `materialized_postgresql_allow_automatic_update` - Allow to reload table in the background, when schema changes are detected. Default: `0` (`false`).
|
||||
|
||||
``` sql
|
||||
CREATE DATABASE test_database
|
||||
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password'
|
||||
SETTINGS materialized_postgresql_max_block_size = 65536,
|
||||
materialized_postgresql_tables_list = 'table1,table2,table3';
|
||||
|
||||
SELECT * FROM test_database.table1;
|
||||
```
|
||||
|
||||
|
||||
## Requirements {#requirements}
|
||||
|
||||
- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file.
|
||||
|
||||
- Each replicated table must have one of the following **replica identity**:
|
||||
|
||||
1. **default** (primary key)
|
||||
|
||||
2. **index**
|
||||
|
||||
``` bash
|
||||
postgres# CREATE TABLE postgres_table (a Integer NOT NULL, b Integer, c Integer NOT NULL, d Integer, e Integer NOT NULL);
|
||||
postgres# CREATE unique INDEX postgres_table_index on postgres_table(a, c, e);
|
||||
postgres# ALTER TABLE postgres_table REPLICA IDENTITY USING INDEX postgres_table_index;
|
||||
```
|
||||
|
||||
|
||||
Primary key is always checked first. If it is absent, then index, defined as replica identity index, is checked.
|
||||
If index is used as replica identity, there has to be only one such index in a table.
|
||||
You can check what type is used for a specific table with the following command:
|
||||
|
||||
``` bash
|
||||
postgres# SELECT CASE relreplident
|
||||
WHEN 'd' THEN 'default'
|
||||
WHEN 'n' THEN 'nothing'
|
||||
WHEN 'f' THEN 'full'
|
||||
WHEN 'i' THEN 'index'
|
||||
END AS replica_identity
|
||||
FROM pg_class
|
||||
WHERE oid = 'postgres_table'::regclass;
|
||||
```
|
||||
|
||||
|
||||
## Warning {#warning}
|
||||
|
||||
1. **TOAST** values convertion is not supported. Default value for the data type will be used.
|
@ -0,0 +1,46 @@
|
||||
---
|
||||
toc_priority: 12
|
||||
toc_title: MateriaziePostgreSQL
|
||||
---
|
||||
|
||||
# MaterializedPostgreSQL {#materialize-postgresql}
|
||||
|
||||
## Creating a Table {#creating-a-table}
|
||||
|
||||
``` sql
|
||||
CREATE TABLE test.postgresql_replica (key UInt64, value UInt64)
|
||||
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password')
|
||||
PRIMARY KEY key;
|
||||
```
|
||||
|
||||
|
||||
## Requirements {#requirements}
|
||||
|
||||
- Setting `wal_level`to `logical` and `max_replication_slots` to at least `2` in the postgresql config file.
|
||||
|
||||
- A table with engine `MaterializedPostgreSQL` must have a primary key - the same as a replica identity index (default: primary key) of a postgres table (See [details on replica identity index](../../database-engines/materialized-postgresql.md#requirements)).
|
||||
|
||||
- Only database `Atomic` is allowed.
|
||||
|
||||
|
||||
## Virtual columns {#creating-a-table}
|
||||
|
||||
- `_version` (`UInt64`)
|
||||
|
||||
- `_sign` (`Int8`)
|
||||
|
||||
These columns do not need to be added, when table is created. They are always accessible in `SELECT` query.
|
||||
`_version` column equals `LSN` position in `WAL`, so it might be used to check how up-to-date replication is.
|
||||
|
||||
``` sql
|
||||
CREATE TABLE test.postgresql_replica (key UInt64, value UInt64)
|
||||
ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgresql_replica', 'postgres_user', 'postgres_password')
|
||||
PRIMARY KEY key;
|
||||
|
||||
SELECT key, value, _version FROM test.postgresql_replica;
|
||||
```
|
||||
|
||||
|
||||
## Warning {#warning}
|
||||
|
||||
1. **TOAST** values convertion is not supported. Default value for the data type will be used.
|
@ -85,6 +85,7 @@ if (USE_AMQPCPP)
|
||||
endif()
|
||||
|
||||
if (USE_LIBPQXX)
|
||||
add_headers_and_sources(dbms Core/PostgreSQL)
|
||||
add_headers_and_sources(dbms Databases/PostgreSQL)
|
||||
add_headers_and_sources(dbms Storages/PostgreSQL)
|
||||
endif()
|
||||
|
73
src/Core/PostgreSQL/Connection.cpp
Normal file
73
src/Core/PostgreSQL/Connection.cpp
Normal file
@ -0,0 +1,73 @@
|
||||
#include "Connection.h"
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
|
||||
Connection::Connection(const ConnectionInfo & connection_info_, bool replication_, size_t num_tries_)
|
||||
: connection_info(connection_info_), replication(replication_), num_tries(num_tries_)
|
||||
, log(&Poco::Logger::get("PostgreSQLReplicaConnection"))
|
||||
{
|
||||
if (replication)
|
||||
{
|
||||
connection_info = std::make_pair(
|
||||
fmt::format("{} replication=database", connection_info.first), connection_info.second);
|
||||
}
|
||||
}
|
||||
|
||||
void Connection::execWithRetry(const std::function<void(pqxx::nontransaction &)> & exec)
|
||||
{
|
||||
for (size_t try_no = 0; try_no < num_tries; ++try_no)
|
||||
{
|
||||
try
|
||||
{
|
||||
pqxx::nontransaction tx(getRef());
|
||||
exec(tx);
|
||||
}
|
||||
catch (const pqxx::broken_connection & e)
|
||||
{
|
||||
LOG_DEBUG(log, "Cannot execute query due to connection failure, attempt: {}/{}. (Message: {})",
|
||||
try_no, num_tries, e.what());
|
||||
|
||||
if (try_no == num_tries)
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pqxx::connection & Connection::getRef()
|
||||
{
|
||||
connect();
|
||||
assert(connection != nullptr);
|
||||
return *connection;
|
||||
}
|
||||
|
||||
void Connection::tryUpdateConnection()
|
||||
{
|
||||
try
|
||||
{
|
||||
updateConnection();
|
||||
}
|
||||
catch (const pqxx::broken_connection & e)
|
||||
{
|
||||
LOG_ERROR(log, "Unable to update connection: {}", e.what());
|
||||
}
|
||||
}
|
||||
|
||||
void Connection::updateConnection()
|
||||
{
|
||||
if (connection)
|
||||
connection->close();
|
||||
/// Always throws if there is no connection.
|
||||
connection = std::make_unique<pqxx::connection>(connection_info.first);
|
||||
if (replication)
|
||||
connection->set_variable("default_transaction_isolation", "'repeatable read'");
|
||||
LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.second);
|
||||
}
|
||||
|
||||
void Connection::connect()
|
||||
{
|
||||
if (!connection || !connection->is_open())
|
||||
updateConnection();
|
||||
}
|
||||
}
|
47
src/Core/PostgreSQL/Connection.h
Normal file
47
src/Core/PostgreSQL/Connection.h
Normal file
@ -0,0 +1,47 @@
|
||||
#pragma once
|
||||
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
#include <Core/Types.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
/* Methods to work with PostgreSQL connection object.
|
||||
* Should only be used in case there has to be a single connection object, which
|
||||
* is long-lived and there are no concurrent connection queries.
|
||||
* Now only use case - for replication handler for replication from PostgreSQL.
|
||||
* In all other integration engine use pool with failover.
|
||||
**/
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
using ConnectionInfo = std::pair<String, String>;
|
||||
using ConnectionPtr = std::unique_ptr<pqxx::connection>;
|
||||
|
||||
class Connection : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
Connection(const ConnectionInfo & connection_info_, bool replication_ = false, size_t num_tries = 3);
|
||||
|
||||
void execWithRetry(const std::function<void(pqxx::nontransaction &)> & exec);
|
||||
|
||||
pqxx::connection & getRef();
|
||||
|
||||
void connect();
|
||||
|
||||
void tryUpdateConnection();
|
||||
|
||||
const ConnectionInfo & getConnectionInfo() { return connection_info; }
|
||||
|
||||
private:
|
||||
void updateConnection();
|
||||
|
||||
ConnectionPtr connection;
|
||||
ConnectionInfo connection_info;
|
||||
|
||||
bool replication;
|
||||
size_t num_tries;
|
||||
|
||||
Poco::Logger * log;
|
||||
};
|
||||
}
|
@ -1,7 +1,7 @@
|
||||
#include <Storages/PostgreSQL/PoolWithFailover.h>
|
||||
#include "PoolWithFailover.h"
|
||||
#include "Utils.h"
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,18 +14,6 @@ namespace ErrorCodes
|
||||
namespace postgres
|
||||
{
|
||||
|
||||
String formatConnectionString(String dbname, String host, UInt16 port, String user, String password)
|
||||
{
|
||||
DB::WriteBufferFromOwnString out;
|
||||
out << "dbname=" << DB::quote << dbname
|
||||
<< " host=" << DB::quote << host
|
||||
<< " port=" << port
|
||||
<< " user=" << DB::quote << user
|
||||
<< " password=" << DB::quote << password
|
||||
<< " connect_timeout=10";
|
||||
return out.str();
|
||||
}
|
||||
|
||||
PoolWithFailover::PoolWithFailover(
|
||||
const Poco::Util::AbstractConfiguration & config, const String & config_prefix,
|
||||
size_t pool_size, size_t pool_wait_timeout_, size_t max_tries_)
|
||||
@ -58,14 +46,14 @@ PoolWithFailover::PoolWithFailover(
|
||||
auto replica_user = config.getString(replica_name + ".user", user);
|
||||
auto replica_password = config.getString(replica_name + ".password", password);
|
||||
|
||||
auto connection_string = formatConnectionString(db, replica_host, replica_port, replica_user, replica_password);
|
||||
auto connection_string = formatConnectionString(db, replica_host, replica_port, replica_user, replica_password).first;
|
||||
replicas_with_priority[priority].emplace_back(connection_string, pool_size);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto connection_string = formatConnectionString(db, host, port, user, password);
|
||||
auto connection_string = formatConnectionString(db, host, port, user, password).first;
|
||||
replicas_with_priority[0].emplace_back(connection_string, pool_size);
|
||||
}
|
||||
}
|
||||
@ -85,7 +73,7 @@ PoolWithFailover::PoolWithFailover(
|
||||
for (const auto & [host, port] : addresses)
|
||||
{
|
||||
LOG_DEBUG(&Poco::Logger::get("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port);
|
||||
auto connection_string = formatConnectionString(database, host, port, user, password);
|
||||
auto connection_string = formatConnectionString(database, host, port, user, password).first;
|
||||
replicas_with_priority[0].emplace_back(connection_string, pool_size);
|
||||
}
|
||||
}
|
@ -1,16 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include "ConnectionHolder.h"
|
||||
#include <mutex>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Storages/PostgreSQL/ConnectionHolder.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
|
||||
String formatConnectionString(String dbname, String host, UInt16 port, String user, String password);
|
||||
|
||||
class PoolWithFailover
|
||||
{
|
||||
|
19
src/Core/PostgreSQL/Utils.cpp
Normal file
19
src/Core/PostgreSQL/Utils.cpp
Normal file
@ -0,0 +1,19 @@
|
||||
#include "Utils.h"
|
||||
#include <IO/Operators.h>
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
|
||||
ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password)
|
||||
{
|
||||
DB::WriteBufferFromOwnString out;
|
||||
out << "dbname=" << DB::quote << dbname
|
||||
<< " host=" << DB::quote << host
|
||||
<< " port=" << port
|
||||
<< " user=" << DB::quote << user
|
||||
<< " password=" << DB::quote << password
|
||||
<< " connect_timeout=10";
|
||||
return std::make_pair(out.str(), host + ':' + DB::toString(port));
|
||||
}
|
||||
|
||||
}
|
17
src/Core/PostgreSQL/Utils.h
Normal file
17
src/Core/PostgreSQL/Utils.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
#include <Core/Types.h>
|
||||
#include "Connection.h"
|
||||
#include <Common/Exception.h>
|
||||
|
||||
namespace pqxx
|
||||
{
|
||||
using ReadTransaction = pqxx::read_transaction;
|
||||
using ReplicationTransaction = pqxx::transaction<isolation_level::repeatable_read, write_policy::read_only>;
|
||||
}
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, String user, String password);
|
||||
}
|
241
src/Core/PostgreSQL/insertPostgreSQLValue.cpp
Normal file
241
src/Core/PostgreSQL/insertPostgreSQLValue.cpp
Normal file
@ -0,0 +1,241 @@
|
||||
#include "insertPostgreSQLValue.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <pqxx/pqxx> // Y_IGNORE
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_column)
|
||||
{
|
||||
column.insertFrom(sample_column, 0);
|
||||
}
|
||||
|
||||
|
||||
void insertPostgreSQLValue(
|
||||
IColumn & column, std::string_view value,
|
||||
const ExternalResultDescription::ValueType type, const DataTypePtr data_type,
|
||||
std::unordered_map<size_t, PostgreSQLArrayInfo> & array_info, size_t idx)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
|
||||
case ExternalResultDescription::ValueType::vtUInt8:
|
||||
{
|
||||
if (value == "t")
|
||||
assert_cast<ColumnUInt8 &>(column).insertValue(1);
|
||||
else if (value == "f")
|
||||
assert_cast<ColumnUInt8 &>(column).insertValue(0);
|
||||
else
|
||||
assert_cast<ColumnUInt8 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
||||
break;
|
||||
}
|
||||
case ExternalResultDescription::ValueType::vtUInt16:
|
||||
assert_cast<ColumnUInt16 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtUInt32:
|
||||
assert_cast<ColumnUInt32 &>(column).insertValue(pqxx::from_string<uint32_t>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtUInt64:
|
||||
assert_cast<ColumnUInt64 &>(column).insertValue(pqxx::from_string<uint64_t>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtInt8:
|
||||
assert_cast<ColumnInt8 &>(column).insertValue(pqxx::from_string<int16_t>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtInt16:
|
||||
assert_cast<ColumnInt16 &>(column).insertValue(pqxx::from_string<int16_t>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtInt32:
|
||||
assert_cast<ColumnInt32 &>(column).insertValue(pqxx::from_string<int32_t>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtInt64:
|
||||
assert_cast<ColumnInt64 &>(column).insertValue(pqxx::from_string<int64_t>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtFloat32:
|
||||
assert_cast<ColumnFloat32 &>(column).insertValue(pqxx::from_string<float>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtFloat64:
|
||||
assert_cast<ColumnFloat64 &>(column).insertValue(pqxx::from_string<double>(value));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtEnum8:[[fallthrough]];
|
||||
case ExternalResultDescription::ValueType::vtEnum16:[[fallthrough]];
|
||||
case ExternalResultDescription::ValueType::vtFixedString:[[fallthrough]];
|
||||
case ExternalResultDescription::ValueType::vtString:
|
||||
assert_cast<ColumnString &>(column).insertData(value.data(), value.size());
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtUUID:
|
||||
assert_cast<ColumnUInt128 &>(column).insert(parse<UUID>(value.data(), value.size()));
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtDate:
|
||||
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()});
|
||||
break;
|
||||
case ExternalResultDescription::ValueType::vtDateTime:
|
||||
{
|
||||
ReadBufferFromString in(value);
|
||||
time_t time = 0;
|
||||
readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(data_type.get())->getTimeZone());
|
||||
if (time < 0)
|
||||
time = 0;
|
||||
assert_cast<ColumnUInt32 &>(column).insertValue(time);
|
||||
break;
|
||||
}
|
||||
case ExternalResultDescription::ValueType::vtDateTime64:[[fallthrough]];
|
||||
case ExternalResultDescription::ValueType::vtDecimal32: [[fallthrough]];
|
||||
case ExternalResultDescription::ValueType::vtDecimal64: [[fallthrough]];
|
||||
case ExternalResultDescription::ValueType::vtDecimal128: [[fallthrough]];
|
||||
case ExternalResultDescription::ValueType::vtDecimal256:
|
||||
{
|
||||
ReadBufferFromString istr(value);
|
||||
data_type->getDefaultSerialization()->deserializeWholeText(column, istr, FormatSettings{});
|
||||
break;
|
||||
}
|
||||
case ExternalResultDescription::ValueType::vtArray:
|
||||
{
|
||||
pqxx::array_parser parser{value};
|
||||
std::pair<pqxx::array_parser::juncture, std::string> parsed = parser.get_next();
|
||||
|
||||
size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions;
|
||||
const auto parse_value = array_info[idx].pqxx_parser;
|
||||
std::vector<Row> dimensions(expected_dimensions + 1);
|
||||
|
||||
while (parsed.first != pqxx::array_parser::juncture::done)
|
||||
{
|
||||
if ((parsed.first == pqxx::array_parser::juncture::row_start) && (++dimension > expected_dimensions))
|
||||
throw Exception("Got more dimensions than expected", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
else if (parsed.first == pqxx::array_parser::juncture::string_value)
|
||||
dimensions[dimension].emplace_back(parse_value(parsed.second));
|
||||
|
||||
else if (parsed.first == pqxx::array_parser::juncture::null_value)
|
||||
dimensions[dimension].emplace_back(array_info[idx].default_value);
|
||||
|
||||
else if (parsed.first == pqxx::array_parser::juncture::row_end)
|
||||
{
|
||||
max_dimension = std::max(max_dimension, dimension);
|
||||
|
||||
--dimension;
|
||||
if (dimension == 0)
|
||||
break;
|
||||
|
||||
dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end()));
|
||||
dimensions[dimension + 1].clear();
|
||||
}
|
||||
|
||||
parsed = parser.get_next();
|
||||
}
|
||||
|
||||
if (max_dimension < expected_dimensions)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Got less dimensions than expected. ({} instead of {})", max_dimension, expected_dimensions);
|
||||
|
||||
assert_cast<ColumnArray &>(column).insert(Array(dimensions[1].begin(), dimensions[1].end()));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void preparePostgreSQLArrayInfo(
|
||||
std::unordered_map<size_t, PostgreSQLArrayInfo> & array_info, size_t column_idx, const DataTypePtr data_type)
|
||||
{
|
||||
const auto * array_type = typeid_cast<const DataTypeArray *>(data_type.get());
|
||||
auto nested = array_type->getNestedType();
|
||||
|
||||
size_t count_dimensions = 1;
|
||||
while (isArray(nested))
|
||||
{
|
||||
++count_dimensions;
|
||||
nested = typeid_cast<const DataTypeArray *>(nested.get())->getNestedType();
|
||||
}
|
||||
|
||||
Field default_value = nested->getDefault();
|
||||
if (nested->isNullable())
|
||||
nested = static_cast<const DataTypeNullable *>(nested.get())->getNestedType();
|
||||
|
||||
WhichDataType which(nested);
|
||||
std::function<Field(std::string & fields)> parser;
|
||||
|
||||
if (which.isUInt8() || which.isUInt16())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<uint16_t>(field); };
|
||||
else if (which.isInt8() || which.isInt16())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<int16_t>(field); };
|
||||
else if (which.isUInt32())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<uint32_t>(field); };
|
||||
else if (which.isInt32())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<int32_t>(field); };
|
||||
else if (which.isUInt64())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<uint64_t>(field); };
|
||||
else if (which.isInt64())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<int64_t>(field); };
|
||||
else if (which.isFloat32())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<float>(field); };
|
||||
else if (which.isFloat64())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<double>(field); };
|
||||
else if (which.isString() || which.isFixedString())
|
||||
parser = [](std::string & field) -> Field { return field; };
|
||||
else if (which.isDate())
|
||||
parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; };
|
||||
else if (which.isDateTime())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
ReadBufferFromString in(field);
|
||||
time_t time = 0;
|
||||
readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(nested.get())->getTimeZone());
|
||||
return time;
|
||||
};
|
||||
else if (which.isDecimal32())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal32> *>(nested.get());
|
||||
DataTypeDecimal<Decimal32> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else if (which.isDecimal64())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal64> *>(nested.get());
|
||||
DataTypeDecimal<Decimal64> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else if (which.isDecimal128())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal128> *>(nested.get());
|
||||
DataTypeDecimal<Decimal128> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else if (which.isDecimal256())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal256> *>(nested.get());
|
||||
DataTypeDecimal<Decimal256> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName());
|
||||
|
||||
array_info[column_idx] = {count_dimensions, default_value, parser};
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
38
src/Core/PostgreSQL/insertPostgreSQLValue.h
Normal file
38
src/Core/PostgreSQL/insertPostgreSQLValue.h
Normal file
@ -0,0 +1,38 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Core/ExternalResultDescription.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct PostgreSQLArrayInfo
|
||||
{
|
||||
size_t num_dimensions;
|
||||
Field default_value;
|
||||
std::function<Field(std::string & field)> pqxx_parser;
|
||||
};
|
||||
|
||||
|
||||
void insertPostgreSQLValue(
|
||||
IColumn & column, std::string_view value,
|
||||
const ExternalResultDescription::ValueType type, const DataTypePtr data_type,
|
||||
std::unordered_map<size_t, PostgreSQLArrayInfo> & array_info, size_t idx);
|
||||
|
||||
void preparePostgreSQLArrayInfo(
|
||||
std::unordered_map<size_t, PostgreSQLArrayInfo> & array_info, size_t column_idx, const DataTypePtr data_type);
|
||||
|
||||
void insertDefaultPostgreSQLValue(IColumn & column, const IColumn & sample_column);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -430,6 +430,7 @@ class IColumn;
|
||||
M(Bool, cast_keep_nullable, false, "CAST operator keep Nullable for result data type", 0) \
|
||||
M(Bool, alter_partition_verbose_result, false, "Output information about affected parts. Currently works only for FREEZE and ATTACH commands.", 0) \
|
||||
M(Bool, allow_experimental_database_materialize_mysql, false, "Allow to create database with Engine=MaterializeMySQL(...).", 0) \
|
||||
M(Bool, allow_experimental_database_materialized_postgresql, false, "Allow to create database with Engine=MaterializedPostgreSQL(...).", 0) \
|
||||
M(Bool, system_events_show_zero_values, false, "Include all metrics, even with zero values", 0) \
|
||||
M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precision are seen as String on ClickHouse's side.", 0) \
|
||||
M(Bool, optimize_trivial_insert_select, true, "Optimize trivial 'INSERT INTO table SELECT ... FROM TABLES' query", 0) \
|
||||
|
@ -22,12 +22,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
PostgreSQLBlockInputStream::PostgreSQLBlockInputStream(
|
||||
template<typename T>
|
||||
PostgreSQLBlockInputStream<T>::PostgreSQLBlockInputStream(
|
||||
postgres::ConnectionHolderPtr connection_holder_,
|
||||
const std::string & query_str_,
|
||||
const Block & sample_block,
|
||||
@ -35,25 +32,52 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream(
|
||||
: query_str(query_str_)
|
||||
, max_block_size(max_block_size_)
|
||||
, connection_holder(std::move(connection_holder_))
|
||||
{
|
||||
init(sample_block);
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
PostgreSQLBlockInputStream<T>::PostgreSQLBlockInputStream(
|
||||
std::shared_ptr<T> tx_,
|
||||
const std::string & query_str_,
|
||||
const Block & sample_block,
|
||||
const UInt64 max_block_size_,
|
||||
bool auto_commit_)
|
||||
: query_str(query_str_)
|
||||
, tx(std::move(tx_))
|
||||
, max_block_size(max_block_size_)
|
||||
, auto_commit(auto_commit_)
|
||||
{
|
||||
init(sample_block);
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
void PostgreSQLBlockInputStream<T>::init(const Block & sample_block)
|
||||
{
|
||||
description.init(sample_block);
|
||||
|
||||
for (const auto idx : collections::range(0, description.sample_block.columns()))
|
||||
if (description.types[idx].first == ValueType::vtArray)
|
||||
prepareArrayInfo(idx, description.sample_block.getByPosition(idx).type);
|
||||
if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray)
|
||||
preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type);
|
||||
|
||||
/// pqxx::stream_from uses COPY command, will get error if ';' is present
|
||||
if (query_str.ends_with(';'))
|
||||
query_str.resize(query_str.size() - 1);
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLBlockInputStream::readPrefix()
|
||||
template<typename T>
|
||||
void PostgreSQLBlockInputStream<T>::readPrefix()
|
||||
{
|
||||
tx = std::make_unique<pqxx::read_transaction>(connection_holder->get());
|
||||
tx = std::make_shared<T>(connection_holder->get());
|
||||
stream = std::make_unique<pqxx::stream_from>(*tx, pqxx::from_query, std::string_view(query_str));
|
||||
}
|
||||
|
||||
|
||||
Block PostgreSQLBlockInputStream::readImpl()
|
||||
template<typename T>
|
||||
Block PostgreSQLBlockInputStream<T>::readImpl()
|
||||
{
|
||||
/// Check if pqxx::stream_from is finished
|
||||
if (!stream || !(*stream))
|
||||
@ -81,17 +105,22 @@ Block PostgreSQLBlockInputStream::readImpl()
|
||||
{
|
||||
ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(*columns[idx]);
|
||||
const auto & data_type = assert_cast<const DataTypeNullable &>(*sample.type);
|
||||
insertValue(column_nullable.getNestedColumn(), (*row)[idx], description.types[idx].first, data_type.getNestedType(), idx);
|
||||
|
||||
insertPostgreSQLValue(
|
||||
column_nullable.getNestedColumn(), (*row)[idx],
|
||||
description.types[idx].first, data_type.getNestedType(), array_info, idx);
|
||||
|
||||
column_nullable.getNullMapData().emplace_back(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
insertValue(*columns[idx], (*row)[idx], description.types[idx].first, sample.type, idx);
|
||||
insertPostgreSQLValue(
|
||||
*columns[idx], (*row)[idx], description.types[idx].first, sample.type, array_info, idx);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
insertDefaultValue(*columns[idx], *sample.column);
|
||||
insertDefaultPostgreSQLValue(*columns[idx], *sample.column);
|
||||
}
|
||||
|
||||
}
|
||||
@ -104,216 +133,23 @@ Block PostgreSQLBlockInputStream::readImpl()
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLBlockInputStream::readSuffix()
|
||||
template<typename T>
|
||||
void PostgreSQLBlockInputStream<T>::readSuffix()
|
||||
{
|
||||
if (stream)
|
||||
{
|
||||
stream->complete();
|
||||
tx->commit();
|
||||
|
||||
if (auto_commit)
|
||||
tx->commit();
|
||||
}
|
||||
}
|
||||
|
||||
template
|
||||
class PostgreSQLBlockInputStream<pqxx::ReplicationTransaction>;
|
||||
|
||||
void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view value,
|
||||
const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case ValueType::vtUInt8:
|
||||
{
|
||||
if (value == "t")
|
||||
assert_cast<ColumnUInt8 &>(column).insertValue(1);
|
||||
else if (value == "f")
|
||||
assert_cast<ColumnUInt8 &>(column).insertValue(0);
|
||||
else
|
||||
assert_cast<ColumnUInt8 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
||||
break;
|
||||
}
|
||||
case ValueType::vtUInt16:
|
||||
assert_cast<ColumnUInt16 &>(column).insertValue(pqxx::from_string<uint16_t>(value));
|
||||
break;
|
||||
case ValueType::vtUInt32:
|
||||
assert_cast<ColumnUInt32 &>(column).insertValue(pqxx::from_string<uint32_t>(value));
|
||||
break;
|
||||
case ValueType::vtUInt64:
|
||||
assert_cast<ColumnUInt64 &>(column).insertValue(pqxx::from_string<uint64_t>(value));
|
||||
break;
|
||||
case ValueType::vtInt8:
|
||||
assert_cast<ColumnInt8 &>(column).insertValue(pqxx::from_string<int16_t>(value));
|
||||
break;
|
||||
case ValueType::vtInt16:
|
||||
assert_cast<ColumnInt16 &>(column).insertValue(pqxx::from_string<int16_t>(value));
|
||||
break;
|
||||
case ValueType::vtInt32:
|
||||
assert_cast<ColumnInt32 &>(column).insertValue(pqxx::from_string<int32_t>(value));
|
||||
break;
|
||||
case ValueType::vtInt64:
|
||||
assert_cast<ColumnInt64 &>(column).insertValue(pqxx::from_string<int64_t>(value));
|
||||
break;
|
||||
case ValueType::vtFloat32:
|
||||
assert_cast<ColumnFloat32 &>(column).insertValue(pqxx::from_string<float>(value));
|
||||
break;
|
||||
case ValueType::vtFloat64:
|
||||
assert_cast<ColumnFloat64 &>(column).insertValue(pqxx::from_string<double>(value));
|
||||
break;
|
||||
case ValueType::vtFixedString:[[fallthrough]];
|
||||
case ValueType::vtEnum8:
|
||||
case ValueType::vtEnum16:
|
||||
case ValueType::vtString:
|
||||
assert_cast<ColumnString &>(column).insertData(value.data(), value.size());
|
||||
break;
|
||||
case ValueType::vtUUID:
|
||||
assert_cast<ColumnUUID &>(column).insert(parse<UUID>(value.data(), value.size()));
|
||||
break;
|
||||
case ValueType::vtDate:
|
||||
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()});
|
||||
break;
|
||||
case ValueType::vtDateTime:
|
||||
{
|
||||
ReadBufferFromString in(value);
|
||||
time_t time = 0;
|
||||
readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(data_type.get())->getTimeZone());
|
||||
if (time < 0)
|
||||
time = 0;
|
||||
assert_cast<ColumnUInt32 &>(column).insertValue(time);
|
||||
break;
|
||||
}
|
||||
case ValueType::vtDateTime64:[[fallthrough]];
|
||||
case ValueType::vtDecimal32: [[fallthrough]];
|
||||
case ValueType::vtDecimal64: [[fallthrough]];
|
||||
case ValueType::vtDecimal128: [[fallthrough]];
|
||||
case ValueType::vtDecimal256:
|
||||
{
|
||||
ReadBufferFromString istr(value);
|
||||
data_type->getDefaultSerialization()->deserializeWholeText(column, istr, FormatSettings{});
|
||||
break;
|
||||
}
|
||||
case ValueType::vtArray:
|
||||
{
|
||||
pqxx::array_parser parser{value};
|
||||
std::pair<pqxx::array_parser::juncture, std::string> parsed = parser.get_next();
|
||||
|
||||
size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions;
|
||||
const auto parse_value = array_info[idx].pqxx_parser;
|
||||
std::vector<Row> dimensions(expected_dimensions + 1);
|
||||
|
||||
while (parsed.first != pqxx::array_parser::juncture::done)
|
||||
{
|
||||
if ((parsed.first == pqxx::array_parser::juncture::row_start) && (++dimension > expected_dimensions))
|
||||
throw Exception("Got more dimensions than expected", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
else if (parsed.first == pqxx::array_parser::juncture::string_value)
|
||||
dimensions[dimension].emplace_back(parse_value(parsed.second));
|
||||
|
||||
else if (parsed.first == pqxx::array_parser::juncture::null_value)
|
||||
dimensions[dimension].emplace_back(array_info[idx].default_value);
|
||||
|
||||
else if (parsed.first == pqxx::array_parser::juncture::row_end)
|
||||
{
|
||||
max_dimension = std::max(max_dimension, dimension);
|
||||
|
||||
--dimension;
|
||||
if (dimension == 0)
|
||||
break;
|
||||
|
||||
dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end()));
|
||||
dimensions[dimension + 1].clear();
|
||||
}
|
||||
|
||||
parsed = parser.get_next();
|
||||
}
|
||||
|
||||
if (max_dimension < expected_dimensions)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Got less dimensions than expected. ({} instead of {})", max_dimension, expected_dimensions);
|
||||
|
||||
assert_cast<ColumnArray &>(column).insert(Array(dimensions[1].begin(), dimensions[1].end()));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLBlockInputStream::prepareArrayInfo(size_t column_idx, const DataTypePtr data_type)
|
||||
{
|
||||
const auto * array_type = typeid_cast<const DataTypeArray *>(data_type.get());
|
||||
auto nested = array_type->getNestedType();
|
||||
|
||||
size_t count_dimensions = 1;
|
||||
while (isArray(nested))
|
||||
{
|
||||
++count_dimensions;
|
||||
nested = typeid_cast<const DataTypeArray *>(nested.get())->getNestedType();
|
||||
}
|
||||
|
||||
Field default_value = nested->getDefault();
|
||||
if (nested->isNullable())
|
||||
nested = static_cast<const DataTypeNullable *>(nested.get())->getNestedType();
|
||||
|
||||
WhichDataType which(nested);
|
||||
std::function<Field(std::string & fields)> parser;
|
||||
|
||||
if (which.isUInt8() || which.isUInt16())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<uint16_t>(field); };
|
||||
else if (which.isInt8() || which.isInt16())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<int16_t>(field); };
|
||||
else if (which.isUInt32())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<uint32_t>(field); };
|
||||
else if (which.isInt32())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<int32_t>(field); };
|
||||
else if (which.isUInt64())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<uint64_t>(field); };
|
||||
else if (which.isInt64())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<int64_t>(field); };
|
||||
else if (which.isFloat32())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<float>(field); };
|
||||
else if (which.isFloat64())
|
||||
parser = [](std::string & field) -> Field { return pqxx::from_string<double>(field); };
|
||||
else if (which.isString() || which.isFixedString())
|
||||
parser = [](std::string & field) -> Field { return field; };
|
||||
else if (which.isDate())
|
||||
parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; };
|
||||
else if (which.isDateTime())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
ReadBufferFromString in(field);
|
||||
time_t time = 0;
|
||||
readDateTimeText(time, in, assert_cast<const DataTypeDateTime *>(nested.get())->getTimeZone());
|
||||
return time;
|
||||
};
|
||||
else if (which.isDecimal32())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal32> *>(nested.get());
|
||||
DataTypeDecimal<Decimal32> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else if (which.isDecimal64())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal64> *>(nested.get());
|
||||
DataTypeDecimal<Decimal64> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else if (which.isDecimal128())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal128> *>(nested.get());
|
||||
DataTypeDecimal<Decimal128> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else if (which.isDecimal256())
|
||||
parser = [nested](std::string & field) -> Field
|
||||
{
|
||||
const auto & type = typeid_cast<const DataTypeDecimal<Decimal256> *>(nested.get());
|
||||
DataTypeDecimal<Decimal256> res(getDecimalPrecision(*type), getDecimalScale(*type));
|
||||
return convertFieldToType(field, res);
|
||||
};
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName());
|
||||
|
||||
array_info[column_idx] = {count_dimensions, default_value, parser};
|
||||
}
|
||||
template
|
||||
class PostgreSQLBlockInputStream<pqxx::ReadTransaction>;
|
||||
|
||||
}
|
||||
|
||||
|
@ -9,54 +9,76 @@
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Core/ExternalResultDescription.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Storages/PostgreSQL/ConnectionHolder.h>
|
||||
#include <Core/PostgreSQL/insertPostgreSQLValue.h>
|
||||
#include <Core/PostgreSQL/ConnectionHolder.h>
|
||||
#include <Core/PostgreSQL/Utils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T = pqxx::ReadTransaction>
|
||||
class PostgreSQLBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
|
||||
public:
|
||||
PostgreSQLBlockInputStream(
|
||||
postgres::ConnectionHolderPtr connection_holder_,
|
||||
const std::string & query_str,
|
||||
const String & query_str_,
|
||||
const Block & sample_block,
|
||||
const UInt64 max_block_size_);
|
||||
|
||||
String getName() const override { return "PostgreSQL"; }
|
||||
Block getHeader() const override { return description.sample_block.cloneEmpty(); }
|
||||
|
||||
private:
|
||||
using ValueType = ExternalResultDescription::ValueType;
|
||||
|
||||
void readPrefix() override;
|
||||
|
||||
protected:
|
||||
PostgreSQLBlockInputStream(
|
||||
std::shared_ptr<T> tx_,
|
||||
const std::string & query_str_,
|
||||
const Block & sample_block,
|
||||
const UInt64 max_block_size_,
|
||||
bool auto_commit_);
|
||||
|
||||
String query_str;
|
||||
std::shared_ptr<T> tx;
|
||||
std::unique_ptr<pqxx::stream_from> stream;
|
||||
|
||||
private:
|
||||
Block readImpl() override;
|
||||
void readSuffix() override;
|
||||
|
||||
void insertValue(IColumn & column, std::string_view value,
|
||||
const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx);
|
||||
void insertDefaultValue(IColumn & column, const IColumn & sample_column)
|
||||
{
|
||||
column.insertFrom(sample_column, 0);
|
||||
}
|
||||
void prepareArrayInfo(size_t column_idx, const DataTypePtr data_type);
|
||||
void init(const Block & sample_block);
|
||||
|
||||
String query_str;
|
||||
const UInt64 max_block_size;
|
||||
bool auto_commit = true;
|
||||
ExternalResultDescription description;
|
||||
|
||||
postgres::ConnectionHolderPtr connection_holder;
|
||||
std::unique_ptr<pqxx::read_transaction> tx;
|
||||
std::unique_ptr<pqxx::stream_from> stream;
|
||||
|
||||
struct ArrayInfo
|
||||
std::unordered_map<size_t, PostgreSQLArrayInfo> array_info;
|
||||
};
|
||||
|
||||
|
||||
/// Passes transaction object into PostgreSQLBlockInputStream and does not close transaction after read is finished.
|
||||
template <typename T>
|
||||
class PostgreSQLTransactionBlockInputStream : public PostgreSQLBlockInputStream<T>
|
||||
{
|
||||
public:
|
||||
using Base = PostgreSQLBlockInputStream<T>;
|
||||
|
||||
PostgreSQLTransactionBlockInputStream(
|
||||
std::shared_ptr<T> tx_,
|
||||
const std::string & query_str_,
|
||||
const Block & sample_block_,
|
||||
const UInt64 max_block_size_)
|
||||
: PostgreSQLBlockInputStream<T>(tx_, query_str_, sample_block_, max_block_size_, false) {}
|
||||
|
||||
void readPrefix() override
|
||||
{
|
||||
size_t num_dimensions;
|
||||
Field default_value;
|
||||
std::function<Field(std::string & field)> pqxx_parser;
|
||||
};
|
||||
std::unordered_map<size_t, ArrayInfo> array_info;
|
||||
Base::stream = std::make_unique<pqxx::stream_from>(*Base::tx, pqxx::from_query, std::string_view(Base::query_str));
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -109,12 +109,11 @@ StoragePtr DatabaseAtomic::detachTable(const String & name)
|
||||
|
||||
void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_name, bool no_delay)
|
||||
{
|
||||
if (auto * mv = dynamic_cast<StorageMaterializedView *>(tryGetTable(table_name, local_context).get()))
|
||||
{
|
||||
/// Remove the inner table (if any) to avoid deadlock
|
||||
/// (due to attempt to execute DROP from the worker thread)
|
||||
mv->dropInnerTable(no_delay, local_context);
|
||||
}
|
||||
auto storage = tryGetTable(table_name, local_context);
|
||||
/// Remove the inner table (if any) to avoid deadlock
|
||||
/// (due to attempt to execute DROP from the worker thread)
|
||||
if (storage)
|
||||
storage->dropInnerTableIfAny(no_delay, local_context);
|
||||
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
String table_metadata_path_drop;
|
||||
@ -568,4 +567,3 @@ void DatabaseAtomic::checkDetachedTableNotInUse(const UUID & uuid)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -36,7 +36,8 @@
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Databases/PostgreSQL/DatabasePostgreSQL.h> // Y_IGNORE
|
||||
#include <Storages/PostgreSQL/PoolWithFailover.h>
|
||||
#include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
#endif
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
@ -99,14 +100,14 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
const UUID & uuid = create.uuid;
|
||||
|
||||
bool engine_may_have_arguments = engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "Lazy" ||
|
||||
engine_name == "Replicated" || engine_name == "PostgreSQL";
|
||||
engine_name == "Replicated" || engine_name == "PostgreSQL" || engine_name == "MaterializedPostgreSQL";
|
||||
if (engine_define->engine->arguments && !engine_may_have_arguments)
|
||||
throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by ||
|
||||
engine_define->primary_key || engine_define->order_by ||
|
||||
engine_define->sample_by;
|
||||
bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated";
|
||||
bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL";
|
||||
if (has_unexpected_element || (!may_have_settings && engine_define->settings))
|
||||
throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings",
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
|
||||
@ -262,6 +263,41 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
||||
return std::make_shared<DatabasePostgreSQL>(
|
||||
context, metadata_path, engine_define, database_name, postgres_database_name, connection_pool, use_table_cache);
|
||||
}
|
||||
else if (engine_name == "MaterializedPostgreSQL")
|
||||
{
|
||||
const ASTFunction * engine = engine_define->engine;
|
||||
|
||||
if (!engine->arguments || engine->arguments->children.size() != 4)
|
||||
{
|
||||
throw Exception(
|
||||
fmt::format("{} Database require host:port, database_name, username, password arguments ", engine_name),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
ASTs & engine_args = engine->arguments->children;
|
||||
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
||||
|
||||
const auto & host_port = safeGetLiteralValue<String>(engine_args[0], engine_name);
|
||||
const auto & postgres_database_name = safeGetLiteralValue<String>(engine_args[1], engine_name);
|
||||
const auto & username = safeGetLiteralValue<String>(engine_args[2], engine_name);
|
||||
const auto & password = safeGetLiteralValue<String>(engine_args[3], engine_name);
|
||||
|
||||
auto parsed_host_port = parseAddress(host_port, 5432);
|
||||
auto connection_info = postgres::formatConnectionString(postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password);
|
||||
|
||||
auto postgresql_replica_settings = std::make_unique<MaterializedPostgreSQLSettings>();
|
||||
|
||||
if (engine_define->settings)
|
||||
postgresql_replica_settings->loadFromQuery(*engine_define);
|
||||
|
||||
return std::make_shared<DatabaseMaterializedPostgreSQL>(
|
||||
context, metadata_path, uuid, engine_define,
|
||||
database_name, postgres_database_name, connection_info,
|
||||
std::move(postgresql_replica_settings));
|
||||
}
|
||||
|
||||
|
||||
#endif
|
||||
|
||||
|
212
src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp
Normal file
212
src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp
Normal file
@ -0,0 +1,212 @@
|
||||
#include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
DatabaseMaterializedPostgreSQL::DatabaseMaterializedPostgreSQL(
|
||||
ContextPtr context_,
|
||||
const String & metadata_path_,
|
||||
UUID uuid_,
|
||||
const ASTStorage * database_engine_define_,
|
||||
const String & database_name_,
|
||||
const String & postgres_database_name,
|
||||
const postgres::ConnectionInfo & connection_info_,
|
||||
std::unique_ptr<MaterializedPostgreSQLSettings> settings_)
|
||||
: DatabaseAtomic(database_name_, metadata_path_, uuid_, "DatabaseMaterializedPostgreSQL (" + database_name_ + ")", context_)
|
||||
, database_engine_define(database_engine_define_->clone())
|
||||
, remote_database_name(postgres_database_name)
|
||||
, connection_info(connection_info_)
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMaterializedPostgreSQL::startSynchronization()
|
||||
{
|
||||
replication_handler = std::make_unique<PostgreSQLReplicationHandler>(
|
||||
/* replication_identifier */database_name,
|
||||
remote_database_name,
|
||||
database_name,
|
||||
connection_info,
|
||||
getContext(),
|
||||
settings->materialized_postgresql_max_block_size.value,
|
||||
settings->materialized_postgresql_allow_automatic_update,
|
||||
/* is_materialized_postgresql_database = */ true,
|
||||
settings->materialized_postgresql_tables_list.value);
|
||||
|
||||
postgres::Connection connection(connection_info);
|
||||
NameSet tables_to_replicate;
|
||||
try
|
||||
{
|
||||
tables_to_replicate = replication_handler->fetchRequiredTables(connection);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_ERROR(log, "Unable to load replicated tables list");
|
||||
throw;
|
||||
}
|
||||
|
||||
if (tables_to_replicate.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of tables to replicate");
|
||||
|
||||
for (const auto & table_name : tables_to_replicate)
|
||||
{
|
||||
/// Check nested ReplacingMergeTree table.
|
||||
auto storage = DatabaseAtomic::tryGetTable(table_name, getContext());
|
||||
|
||||
if (storage)
|
||||
{
|
||||
/// Nested table was already created and synchronized.
|
||||
storage = StorageMaterializedPostgreSQL::create(storage, getContext());
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Nested table does not exist and will be created by replication thread.
|
||||
storage = StorageMaterializedPostgreSQL::create(StorageID(database_name, table_name), getContext());
|
||||
}
|
||||
|
||||
/// Cache MaterializedPostgreSQL wrapper over nested table.
|
||||
materialized_tables[table_name] = storage;
|
||||
|
||||
/// Let replication thread know, which tables it needs to keep in sync.
|
||||
replication_handler->addStorage(table_name, storage->as<StorageMaterializedPostgreSQL>());
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Loaded {} tables. Starting synchronization", materialized_tables.size());
|
||||
replication_handler->startup();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMaterializedPostgreSQL::loadStoredObjects(ContextMutablePtr local_context, bool has_force_restore_data_flag, bool force_attach)
|
||||
{
|
||||
DatabaseAtomic::loadStoredObjects(local_context, has_force_restore_data_flag, force_attach);
|
||||
|
||||
try
|
||||
{
|
||||
startSynchronization();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Cannot load nested database objects for PostgreSQL database engine.");
|
||||
|
||||
if (!force_attach)
|
||||
throw;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
StoragePtr DatabaseMaterializedPostgreSQL::tryGetTable(const String & name, ContextPtr local_context) const
|
||||
{
|
||||
/// In otder to define which table access is needed - to MaterializedPostgreSQL table (only in case of SELECT queries) or
|
||||
/// to its nested ReplacingMergeTree table (in all other cases), the context of a query os modified.
|
||||
/// Also if materialzied_tables set is empty - it means all access is done to ReplacingMergeTree tables - it is a case after
|
||||
/// replication_handler was shutdown.
|
||||
if (local_context->isInternalQuery() || materialized_tables.empty())
|
||||
{
|
||||
return DatabaseAtomic::tryGetTable(name, local_context);
|
||||
}
|
||||
|
||||
/// Note: In select query we call MaterializedPostgreSQL table and it calls tryGetTable from its nested.
|
||||
/// So the only point, where synchronization is needed - access to MaterializedPostgreSQL table wrapper over nested table.
|
||||
std::lock_guard lock(tables_mutex);
|
||||
auto table = materialized_tables.find(name);
|
||||
|
||||
/// Return wrapper over ReplacingMergeTree table. If table synchronization just started, table will not
|
||||
/// be accessible immediately. Table is considered to exist once its nested table was created.
|
||||
if (table != materialized_tables.end() && table->second->as <StorageMaterializedPostgreSQL>()->hasNested())
|
||||
{
|
||||
return table->second;
|
||||
}
|
||||
|
||||
return StoragePtr{};
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMaterializedPostgreSQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & table, const ASTPtr & query)
|
||||
{
|
||||
/// Create table query can only be called from replication thread.
|
||||
if (local_context->isInternalQuery())
|
||||
{
|
||||
DatabaseAtomic::createTable(local_context, table_name, table, query);
|
||||
return;
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Create table query allowed only for ReplacingMergeTree engine and from synchronization thread");
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMaterializedPostgreSQL::shutdown()
|
||||
{
|
||||
stopReplication();
|
||||
DatabaseAtomic::shutdown();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMaterializedPostgreSQL::stopReplication()
|
||||
{
|
||||
if (replication_handler)
|
||||
replication_handler->shutdown();
|
||||
|
||||
/// Clear wrappers over nested, all access is not done to nested tables directly.
|
||||
materialized_tables.clear();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMaterializedPostgreSQL::dropTable(ContextPtr local_context, const String & table_name, bool no_delay)
|
||||
{
|
||||
/// Modify context into nested_context and pass query to Atomic database.
|
||||
DatabaseAtomic::dropTable(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), table_name, no_delay);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMaterializedPostgreSQL::drop(ContextPtr local_context)
|
||||
{
|
||||
if (replication_handler)
|
||||
replication_handler->shutdownFinal();
|
||||
|
||||
DatabaseAtomic::drop(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context));
|
||||
}
|
||||
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseMaterializedPostgreSQL::getTablesIterator(
|
||||
ContextPtr local_context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
/// Modify context into nested_context and pass query to Atomic database.
|
||||
return DatabaseAtomic::getTablesIterator(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), filter_by_table_name);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
#endif
|
77
src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h
Normal file
77
src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h
Normal file
@ -0,0 +1,77 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
|
||||
#include <Storages/PostgreSQL/PostgreSQLReplicationHandler.h>
|
||||
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
||||
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class PostgreSQLConnection;
|
||||
using PostgreSQLConnectionPtr = std::shared_ptr<PostgreSQLConnection>;
|
||||
|
||||
|
||||
class DatabaseMaterializedPostgreSQL : public DatabaseAtomic
|
||||
{
|
||||
|
||||
public:
|
||||
DatabaseMaterializedPostgreSQL(
|
||||
ContextPtr context_,
|
||||
const String & metadata_path_,
|
||||
UUID uuid_,
|
||||
const ASTStorage * database_engine_define_,
|
||||
const String & database_name_,
|
||||
const String & postgres_database_name,
|
||||
const postgres::ConnectionInfo & connection_info,
|
||||
std::unique_ptr<MaterializedPostgreSQLSettings> settings_);
|
||||
|
||||
String getEngineName() const override { return "MaterializedPostgreSQL"; }
|
||||
|
||||
String getMetadataPath() const override { return metadata_path; }
|
||||
|
||||
void loadStoredObjects(ContextMutablePtr, bool, bool force_attach) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(
|
||||
ContextPtr context, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
StoragePtr tryGetTable(const String & name, ContextPtr context) const override;
|
||||
|
||||
void createTable(ContextPtr context, const String & name, const StoragePtr & table, const ASTPtr & query) override;
|
||||
|
||||
void dropTable(ContextPtr local_context, const String & name, bool no_delay) override;
|
||||
|
||||
void drop(ContextPtr local_context) override;
|
||||
|
||||
void stopReplication();
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
private:
|
||||
void startSynchronization();
|
||||
|
||||
ASTPtr database_engine_define;
|
||||
String remote_database_name;
|
||||
postgres::ConnectionInfo connection_info;
|
||||
std::unique_ptr<MaterializedPostgreSQLSettings> settings;
|
||||
|
||||
std::shared_ptr<PostgreSQLReplicationHandler> replication_handler;
|
||||
std::map<std::string, StoragePtr> materialized_tables;
|
||||
mutable std::mutex tables_mutex;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -40,14 +40,14 @@ DatabasePostgreSQL::DatabasePostgreSQL(
|
||||
const ASTStorage * database_engine_define_,
|
||||
const String & dbname_,
|
||||
const String & postgres_dbname,
|
||||
postgres::PoolWithFailoverPtr connection_pool_,
|
||||
const bool cache_tables_)
|
||||
postgres::PoolWithFailoverPtr pool_,
|
||||
bool cache_tables_)
|
||||
: IDatabase(dbname_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, metadata_path(metadata_path_)
|
||||
, database_engine_define(database_engine_define_->clone())
|
||||
, dbname(postgres_dbname)
|
||||
, connection_pool(std::move(connection_pool_))
|
||||
, pool(std::move(pool_))
|
||||
, cache_tables(cache_tables_)
|
||||
{
|
||||
cleaner_task = getContext()->getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); });
|
||||
@ -59,7 +59,8 @@ bool DatabasePostgreSQL::empty() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
auto tables_list = fetchTablesList();
|
||||
auto connection_holder = pool->get();
|
||||
auto tables_list = fetchPostgreSQLTablesList(connection_holder->get());
|
||||
|
||||
for (const auto & table_name : tables_list)
|
||||
if (!detached_or_dropped.count(table_name))
|
||||
@ -74,7 +75,8 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
Tables tables;
|
||||
auto table_names = fetchTablesList();
|
||||
auto connection_holder = pool->get();
|
||||
auto table_names = fetchPostgreSQLTablesList(connection_holder->get());
|
||||
|
||||
for (const auto & table_name : table_names)
|
||||
if (!detached_or_dropped.count(table_name))
|
||||
@ -84,21 +86,6 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator(ContextPtr local
|
||||
}
|
||||
|
||||
|
||||
std::unordered_set<std::string> DatabasePostgreSQL::fetchTablesList() const
|
||||
{
|
||||
std::unordered_set<std::string> tables;
|
||||
std::string query = "SELECT tablename FROM pg_catalog.pg_tables "
|
||||
"WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'";
|
||||
auto connection_holder = connection_pool->get();
|
||||
pqxx::read_transaction tx(connection_holder->get());
|
||||
|
||||
for (auto table_name : tx.stream<std::string>(query))
|
||||
tables.insert(std::get<0>(table_name));
|
||||
|
||||
return tables;
|
||||
}
|
||||
|
||||
|
||||
bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const
|
||||
{
|
||||
if (table_name.find('\'') != std::string::npos
|
||||
@ -108,7 +95,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const
|
||||
"PostgreSQL table name cannot contain single quote or backslash characters, passed {}", table_name);
|
||||
}
|
||||
|
||||
auto connection_holder = connection_pool->get();
|
||||
auto connection_holder = pool->get();
|
||||
pqxx::nontransaction tx(connection_holder->get());
|
||||
|
||||
try
|
||||
@ -163,20 +150,15 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr
|
||||
if (!table_checked && !checkPostgresTable(table_name))
|
||||
return StoragePtr{};
|
||||
|
||||
auto use_nulls = local_context->getSettingsRef().external_table_functions_use_nulls;
|
||||
auto columns = fetchPostgreSQLTableStructure(connection_pool->get(), doubleQuoteString(table_name), use_nulls);
|
||||
auto connection_holder = pool->get();
|
||||
auto columns = fetchPostgreSQLTableStructure(connection_holder->get(), doubleQuoteString(table_name)).columns;
|
||||
|
||||
if (!columns)
|
||||
return StoragePtr{};
|
||||
|
||||
auto storage = StoragePostgreSQL::create(
|
||||
StorageID(database_name, table_name),
|
||||
connection_pool,
|
||||
table_name,
|
||||
ColumnsDescription{*columns},
|
||||
ConstraintsDescription{},
|
||||
String{},
|
||||
local_context);
|
||||
StorageID(database_name, table_name), pool, table_name,
|
||||
ColumnsDescription{*columns}, ConstraintsDescription{}, String{}, local_context);
|
||||
|
||||
if (cache_tables)
|
||||
cached_tables[table_name] = storage;
|
||||
@ -298,7 +280,8 @@ void DatabasePostgreSQL::loadStoredObjects(ContextMutablePtr /* context */, bool
|
||||
void DatabasePostgreSQL::removeOutdatedTables()
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
auto actual_tables = fetchTablesList();
|
||||
auto connection_holder = pool->get();
|
||||
auto actual_tables = fetchPostgreSQLTablesList(connection_holder->get());
|
||||
|
||||
if (cache_tables)
|
||||
{
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Storages/PostgreSQL/PoolWithFailover.h>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -33,7 +33,7 @@ public:
|
||||
const ASTStorage * database_engine_define,
|
||||
const String & dbname_,
|
||||
const String & postgres_dbname,
|
||||
postgres::PoolWithFailoverPtr connection_pool_,
|
||||
postgres::PoolWithFailoverPtr pool_,
|
||||
bool cache_tables_);
|
||||
|
||||
String getEngineName() const override { return "PostgreSQL"; }
|
||||
@ -70,7 +70,7 @@ private:
|
||||
String metadata_path;
|
||||
ASTPtr database_engine_define;
|
||||
String dbname;
|
||||
postgres::PoolWithFailoverPtr connection_pool;
|
||||
postgres::PoolWithFailoverPtr pool;
|
||||
const bool cache_tables;
|
||||
|
||||
mutable Tables cached_tables;
|
||||
@ -78,9 +78,11 @@ private:
|
||||
BackgroundSchedulePool::TaskHolder cleaner_task;
|
||||
|
||||
bool checkPostgresTable(const String & table_name) const;
|
||||
std::unordered_set<std::string> fetchTablesList() const;
|
||||
StoragePtr fetchTable(const String & table_name, ContextPtr context, bool table_checked) const;
|
||||
|
||||
StoragePtr fetchTable(const String & table_name, ContextPtr context, const bool table_checked) const;
|
||||
|
||||
void removeOutdatedTables();
|
||||
|
||||
ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const;
|
||||
};
|
||||
|
||||
|
@ -12,7 +12,8 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <boost/algorithm/string/split.hpp>
|
||||
#include <boost/algorithm/string/trim.hpp>
|
||||
#include <pqxx/pqxx>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Core/PostgreSQL/Utils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -25,7 +26,21 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr convertPostgreSQLDataType(String & type, bool is_nullable, uint16_t dimensions, const std::function<void()> & recheck_array)
|
||||
template<typename T>
|
||||
std::unordered_set<std::string> fetchPostgreSQLTablesList(T & tx)
|
||||
{
|
||||
std::unordered_set<std::string> tables;
|
||||
std::string query = "SELECT tablename FROM pg_catalog.pg_tables "
|
||||
"WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'";
|
||||
|
||||
for (auto table_name : tx.template stream<std::string>(query))
|
||||
tables.insert(std::get<0>(table_name));
|
||||
|
||||
return tables;
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr convertPostgreSQLDataType(String & type, const std::function<void()> & recheck_array, bool is_nullable = false, uint16_t dimensions = 0)
|
||||
{
|
||||
DataTypePtr res;
|
||||
bool is_array = false;
|
||||
@ -116,52 +131,51 @@ static DataTypePtr convertPostgreSQLDataType(String & type, bool is_nullable, ui
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||
postgres::ConnectionHolderPtr connection_holder, const String & postgres_table_name, bool use_nulls)
|
||||
template<typename T>
|
||||
std::shared_ptr<NamesAndTypesList> readNamesAndTypesList(
|
||||
T & tx, const String & postgres_table_name, const String & query, bool use_nulls, bool only_names_and_types)
|
||||
{
|
||||
auto columns = NamesAndTypes();
|
||||
|
||||
if (postgres_table_name.find('\'') != std::string::npos
|
||||
|| postgres_table_name.find('\\') != std::string::npos)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL table name cannot contain single quote or backslash characters, passed {}",
|
||||
postgres_table_name);
|
||||
}
|
||||
|
||||
std::string query = fmt::format(
|
||||
"SELECT attname AS name, format_type(atttypid, atttypmod) AS type, "
|
||||
"attnotnull AS not_null, attndims AS dims "
|
||||
"FROM pg_attribute "
|
||||
"WHERE attrelid = '{}'::regclass "
|
||||
"AND NOT attisdropped AND attnum > 0", postgres_table_name);
|
||||
try
|
||||
{
|
||||
std::set<size_t> recheck_arrays_indexes;
|
||||
{
|
||||
pqxx::read_transaction tx(connection_holder->get());
|
||||
auto stream{pqxx::stream_from::query(tx, query)};
|
||||
|
||||
std::tuple<std::string, std::string, std::string, uint16_t> row;
|
||||
size_t i = 0;
|
||||
auto recheck_array = [&]() { recheck_arrays_indexes.insert(i); };
|
||||
while (stream >> row)
|
||||
|
||||
if (only_names_and_types)
|
||||
{
|
||||
auto data_type = convertPostgreSQLDataType(std::get<1>(row),
|
||||
use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false, i.e. value is nullable
|
||||
std::get<3>(row),
|
||||
recheck_array);
|
||||
columns.push_back(NameAndTypePair(std::get<0>(row), data_type));
|
||||
++i;
|
||||
std::tuple<std::string, std::string> row;
|
||||
while (stream >> row)
|
||||
{
|
||||
columns.push_back(NameAndTypePair(std::get<0>(row), convertPostgreSQLDataType(std::get<1>(row), recheck_array)));
|
||||
++i;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
std::tuple<std::string, std::string, std::string, uint16_t> row;
|
||||
while (stream >> row)
|
||||
{
|
||||
auto data_type = convertPostgreSQLDataType(std::get<1>(row),
|
||||
recheck_array,
|
||||
use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false, i.e. value is nullable
|
||||
std::get<3>(row));
|
||||
columns.push_back(NameAndTypePair(std::get<0>(row), data_type));
|
||||
++i;
|
||||
}
|
||||
}
|
||||
|
||||
stream.complete();
|
||||
tx.commit();
|
||||
}
|
||||
|
||||
for (const auto & i : recheck_arrays_indexes)
|
||||
{
|
||||
const auto & name_and_type = columns[i];
|
||||
|
||||
pqxx::nontransaction tx(connection_holder->get());
|
||||
/// All rows must contain the same number of dimensions, so limit 1 is ok. If number of dimensions in all rows is not the same -
|
||||
/// such arrays are not able to be used as ClickHouse Array at all.
|
||||
pqxx::result result{tx.exec(fmt::format("SELECT array_ndims({}) FROM {} LIMIT 1", name_and_type.name, postgres_table_name))};
|
||||
@ -178,9 +192,7 @@ std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||
|
||||
catch (const pqxx::undefined_table &)
|
||||
{
|
||||
throw Exception(fmt::format(
|
||||
"PostgreSQL table {}.{} does not exist",
|
||||
connection_holder->get().dbname(), postgres_table_name), ErrorCodes::UNKNOWN_TABLE);
|
||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "PostgreSQL table {} does not exist", postgres_table_name);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -188,12 +200,101 @@ std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||
throw;
|
||||
}
|
||||
|
||||
if (columns.empty())
|
||||
return nullptr;
|
||||
|
||||
return std::make_shared<NamesAndTypesList>(NamesAndTypesList(columns.begin(), columns.end()));
|
||||
return !columns.empty() ? std::make_shared<NamesAndTypesList>(columns.begin(), columns.end()) : nullptr;
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
T & tx, const String & postgres_table_name, bool use_nulls, bool with_primary_key, bool with_replica_identity_index)
|
||||
{
|
||||
PostgreSQLTableStructure table;
|
||||
|
||||
std::string query = fmt::format(
|
||||
"SELECT attname AS name, format_type(atttypid, atttypmod) AS type, "
|
||||
"attnotnull AS not_null, attndims AS dims "
|
||||
"FROM pg_attribute "
|
||||
"WHERE attrelid = {}::regclass "
|
||||
"AND NOT attisdropped AND attnum > 0", quoteString(postgres_table_name));
|
||||
|
||||
table.columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, false);
|
||||
|
||||
if (with_primary_key)
|
||||
{
|
||||
/// wiki.postgresql.org/wiki/Retrieve_primary_key_columns
|
||||
query = fmt::format(
|
||||
"SELECT a.attname, format_type(a.atttypid, a.atttypmod) AS data_type "
|
||||
"FROM pg_index i "
|
||||
"JOIN pg_attribute a ON a.attrelid = i.indrelid "
|
||||
"AND a.attnum = ANY(i.indkey) "
|
||||
"WHERE i.indrelid = {}::regclass AND i.indisprimary", quoteString(postgres_table_name));
|
||||
|
||||
table.primary_key_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true);
|
||||
}
|
||||
|
||||
if (with_replica_identity_index && !table.primary_key_columns)
|
||||
{
|
||||
query = fmt::format(
|
||||
"SELECT "
|
||||
"a.attname AS column_name, " /// column name
|
||||
"format_type(a.atttypid, a.atttypmod) as type " /// column type
|
||||
"FROM "
|
||||
"pg_class t, "
|
||||
"pg_class i, "
|
||||
"pg_index ix, "
|
||||
"pg_attribute a "
|
||||
"WHERE "
|
||||
"t.oid = ix.indrelid "
|
||||
"and i.oid = ix.indexrelid "
|
||||
"and a.attrelid = t.oid "
|
||||
"and a.attnum = ANY(ix.indkey) "
|
||||
"and t.relkind = 'r' " /// simple tables
|
||||
"and t.relname = {} " /// Connection is already done to a needed database, only table name is needed.
|
||||
"and ix.indisreplident = 't' " /// index is is replica identity index
|
||||
"ORDER BY a.attname", /// column names
|
||||
quoteString(postgres_table_name));
|
||||
|
||||
table.replica_identity_columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, true);
|
||||
}
|
||||
|
||||
return table;
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLTableStructure fetchPostgreSQLTableStructure(pqxx::connection & connection, const String & postgres_table_name, bool use_nulls)
|
||||
{
|
||||
pqxx::ReadTransaction tx(connection);
|
||||
auto result = fetchPostgreSQLTableStructure(tx, postgres_table_name, use_nulls, false, false);
|
||||
tx.commit();
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
std::unordered_set<std::string> fetchPostgreSQLTablesList(pqxx::connection & connection)
|
||||
{
|
||||
pqxx::ReadTransaction tx(connection);
|
||||
auto result = fetchPostgreSQLTablesList(tx);
|
||||
tx.commit();
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
template
|
||||
PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
pqxx::ReadTransaction & tx, const String & postgres_table_name, bool use_nulls,
|
||||
bool with_primary_key, bool with_replica_identity_index);
|
||||
|
||||
template
|
||||
PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
pqxx::ReplicationTransaction & tx, const String & postgres_table_name, bool use_nulls,
|
||||
bool with_primary_key, bool with_replica_identity_index);
|
||||
|
||||
template
|
||||
std::unordered_set<std::string> fetchPostgreSQLTablesList(pqxx::work & tx);
|
||||
|
||||
template
|
||||
std::unordered_set<std::string> fetchPostgreSQLTablesList(pqxx::ReadTransaction & tx);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -5,15 +5,34 @@
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Storages/PostgreSQL/ConnectionHolder.h>
|
||||
#include <Core/PostgreSQL/ConnectionHolder.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::shared_ptr<NamesAndTypesList> fetchPostgreSQLTableStructure(
|
||||
postgres::ConnectionHolderPtr connection_holder, const String & postgres_table_name, bool use_nulls);
|
||||
struct PostgreSQLTableStructure
|
||||
{
|
||||
std::shared_ptr<NamesAndTypesList> columns = nullptr;
|
||||
std::shared_ptr<NamesAndTypesList> primary_key_columns = nullptr;
|
||||
std::shared_ptr<NamesAndTypesList> replica_identity_columns = nullptr;
|
||||
};
|
||||
|
||||
using PostgreSQLTableStructurePtr = std::unique_ptr<PostgreSQLTableStructure>;
|
||||
|
||||
std::unordered_set<std::string> fetchPostgreSQLTablesList(pqxx::connection & connection);
|
||||
|
||||
PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
pqxx::connection & connection, const String & postgres_table_name, bool use_nulls = true);
|
||||
|
||||
template<typename T>
|
||||
PostgreSQLTableStructure fetchPostgreSQLTableStructure(
|
||||
T & tx, const String & postgres_table_name, bool use_nulls = true,
|
||||
bool with_primary_key = false, bool with_replica_identity_index = false);
|
||||
|
||||
template<typename T>
|
||||
std::unordered_set<std::string> fetchPostgreSQLTablesList(T & tx);
|
||||
|
||||
}
|
||||
|
||||
|
@ -107,9 +107,10 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_col
|
||||
|
||||
BlockInputStreamPtr PostgreSQLDictionarySource::loadBase(const String & query)
|
||||
{
|
||||
return std::make_shared<PostgreSQLBlockInputStream>(pool->get(), query, sample_block, max_block_size);
|
||||
return std::make_shared<PostgreSQLBlockInputStream<>>(pool->get(), query, sample_block, max_block_size);
|
||||
}
|
||||
|
||||
|
||||
bool PostgreSQLDictionarySource::isModified() const
|
||||
{
|
||||
if (!configuration.invalidate_query.empty())
|
||||
@ -128,7 +129,7 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re
|
||||
Block invalidate_sample_block;
|
||||
ColumnPtr column(ColumnString::create());
|
||||
invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared<DataTypeString>(), "Sample Block"));
|
||||
PostgreSQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, 1);
|
||||
PostgreSQLBlockInputStream<> block_input_stream(pool->get(), request, invalidate_sample_block, 1);
|
||||
return readInvalidateQuery(block_input_stream);
|
||||
}
|
||||
|
||||
|
@ -11,8 +11,7 @@
|
||||
#include <Core/Block.h>
|
||||
#include <common/LocalDateTime.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Storages/PostgreSQL/PoolWithFailover.h>
|
||||
#include <pqxx/pqxx>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -267,6 +267,9 @@ private:
|
||||
/// XXX: move this stuff to shared part instead.
|
||||
ContextMutablePtr buffer_context; /// Buffer context. Could be equal to this.
|
||||
|
||||
/// A flag, used to distinguish between user query and internal query to a database engine (MaterializePostgreSQL).
|
||||
bool is_internal_query = false;
|
||||
|
||||
public:
|
||||
// Top-level OpenTelemetry trace context for the query. Makes sense only for a query context.
|
||||
OpenTelemetryTraceContext query_trace_context;
|
||||
@ -742,6 +745,9 @@ public:
|
||||
|
||||
void shutdown();
|
||||
|
||||
bool isInternalQuery() const { return is_internal_query; }
|
||||
void setInternalQuery(bool internal) { is_internal_query = internal; }
|
||||
|
||||
ActionLocksManagerPtr getActionLocksManager();
|
||||
|
||||
enum class ApplicationType
|
||||
|
@ -28,6 +28,10 @@
|
||||
# include <Storages/StorageMaterializeMySQL.h>
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
# include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#endif
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace CurrentMetrics
|
||||
@ -234,6 +238,13 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
return {};
|
||||
}
|
||||
|
||||
#if USE_LIBPQXX
|
||||
if (!context_->isInternalQuery() && (db_and_table.first->getEngineName() == "MaterializedPostgreSQL"))
|
||||
{
|
||||
db_and_table.second = std::make_shared<StorageMaterializedPostgreSQL>(std::move(db_and_table.second), getContext());
|
||||
}
|
||||
#endif
|
||||
|
||||
#if USE_MYSQL
|
||||
/// It's definitely not the best place for this logic, but behaviour must be consistent with DatabaseMaterializeMySQL::tryGetTable(...)
|
||||
if (db_and_table.first->getEngineName() == "MaterializeMySQL")
|
||||
@ -245,6 +256,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
return db_and_table;
|
||||
}
|
||||
|
||||
|
||||
if (table_id.database_name == TEMPORARY_DATABASE)
|
||||
{
|
||||
/// For temporary tables UUIDs are set in Context::resolveStorageID(...).
|
||||
|
@ -151,7 +151,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage));
|
||||
}
|
||||
|
||||
if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated")
|
||||
if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated" || create.storage->engine->name == "MaterializedPostgreSQL")
|
||||
{
|
||||
if (create.attach && create.uuid == UUIDHelpers::Nil)
|
||||
throw Exception(ErrorCodes::INCORRECT_QUERY, "UUID must be specified for ATTACH. "
|
||||
@ -217,6 +217,12 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
"Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
}
|
||||
|
||||
if (create.storage->engine->name == "MaterializedPostgreSQL" && !getContext()->getSettingsRef().allow_experimental_database_materialized_postgresql && !internal)
|
||||
{
|
||||
throw Exception("MaterializedPostgreSQL is an experimental database engine. "
|
||||
"Enable allow_experimental_database_postgresql_replica to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
}
|
||||
|
||||
DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", getContext());
|
||||
|
||||
if (create.uuid != UUIDHelpers::Nil)
|
||||
|
@ -20,6 +20,9 @@
|
||||
# include <Databases/MySQL/DatabaseMaterializeMySQL.h>
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
# include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -317,6 +320,10 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
|
||||
#endif
|
||||
if (auto * replicated = typeid_cast<DatabaseReplicated *>(database.get()))
|
||||
replicated->stopReplication();
|
||||
#if USE_LIBPQXX
|
||||
if (auto * materialize_postgresql = typeid_cast<DatabaseMaterializedPostgreSQL *>(database.get()))
|
||||
materialize_postgresql->stopReplication();
|
||||
#endif
|
||||
|
||||
if (database->shouldBeEmptyOnDetach())
|
||||
{
|
||||
@ -398,4 +405,33 @@ void InterpreterDropQuery::extendQueryLogElemImpl(QueryLogElement & elem, const
|
||||
elem.query_kind = "Drop";
|
||||
}
|
||||
|
||||
void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay)
|
||||
{
|
||||
if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context))
|
||||
{
|
||||
/// We create and execute `drop` query for internal table.
|
||||
auto drop_query = std::make_shared<ASTDropQuery>();
|
||||
drop_query->database = target_table_id.database_name;
|
||||
drop_query->table = target_table_id.table_name;
|
||||
drop_query->kind = kind;
|
||||
drop_query->no_delay = no_delay;
|
||||
drop_query->if_exists = true;
|
||||
ASTPtr ast_drop_query = drop_query;
|
||||
/// FIXME We have to use global context to execute DROP query for inner table
|
||||
/// to avoid "Not enough privileges" error if current user has only DROP VIEW ON mat_view_name privilege
|
||||
/// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant
|
||||
/// looks like expected behaviour and we have tests for it.
|
||||
auto drop_context = Context::createCopy(global_context);
|
||||
drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
if (auto txn = current_context->getZooKeeperMetadataTransaction())
|
||||
{
|
||||
/// For Replicated database
|
||||
drop_context->setQueryContext(std::const_pointer_cast<Context>(current_context));
|
||||
drop_context->initZooKeeperMetadataTransaction(txn, true);
|
||||
}
|
||||
InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context);
|
||||
drop_interpreter.execute();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -26,6 +26,8 @@ public:
|
||||
|
||||
void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override;
|
||||
|
||||
static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay);
|
||||
|
||||
private:
|
||||
AccessRightsElements getRequiredAccessForDDLOnCluster() const;
|
||||
ASTPtr query_ptr;
|
||||
|
@ -51,7 +51,10 @@ TableLockHolder IStorage::lockForShare(const String & query_id, const std::chron
|
||||
TableLockHolder result = tryLockTimed(drop_lock, RWLockImpl::Read, query_id, acquire_timeout);
|
||||
|
||||
if (is_dropped)
|
||||
throw Exception("Table is dropped", ErrorCodes::TABLE_IS_DROPPED);
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
throw Exception(ErrorCodes::TABLE_IS_DROPPED, "Table {}.{} is dropped", table_id.database_name, table_id.table_name);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
@ -351,6 +351,8 @@ public:
|
||||
*/
|
||||
virtual void drop() {}
|
||||
|
||||
virtual void dropInnerTableIfAny(bool /* no_delay */, ContextPtr /* context */) {}
|
||||
|
||||
/** Clear the table data and leave it empty.
|
||||
* Must be called under exclusive lock (lockExclusively).
|
||||
*/
|
||||
|
720
src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp
Normal file
720
src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp
Normal file
@ -0,0 +1,720 @@
|
||||
#include "MaterializedPostgreSQLConsumer.h"
|
||||
|
||||
#include "StorageMaterializedPostgreSQL.h"
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Common/hex.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer(
|
||||
ContextPtr context_,
|
||||
std::shared_ptr<postgres::Connection> connection_,
|
||||
const std::string & replication_slot_name_,
|
||||
const std::string & publication_name_,
|
||||
const std::string & start_lsn,
|
||||
const size_t max_block_size_,
|
||||
bool allow_automatic_update_,
|
||||
Storages storages_)
|
||||
: log(&Poco::Logger::get("PostgreSQLReaplicaConsumer"))
|
||||
, context(context_)
|
||||
, replication_slot_name(replication_slot_name_)
|
||||
, publication_name(publication_name_)
|
||||
, connection(connection_)
|
||||
, current_lsn(start_lsn)
|
||||
, lsn_value(getLSNValue(start_lsn))
|
||||
, max_block_size(max_block_size_)
|
||||
, allow_automatic_update(allow_automatic_update_)
|
||||
, storages(storages_)
|
||||
{
|
||||
final_lsn = start_lsn;
|
||||
auto tx = std::make_shared<pqxx::nontransaction>(connection->getRef());
|
||||
current_lsn = advanceLSN(tx);
|
||||
LOG_TRACE(log, "Starting replication. LSN: {} (last: {})", getLSNValue(current_lsn), getLSNValue(final_lsn));
|
||||
tx->commit();
|
||||
|
||||
for (const auto & [table_name, storage] : storages)
|
||||
{
|
||||
buffers.emplace(table_name, Buffer(storage));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::Buffer::createEmptyBuffer(StoragePtr storage)
|
||||
{
|
||||
const auto storage_metadata = storage->getInMemoryMetadataPtr();
|
||||
const Block sample_block = storage_metadata->getSampleBlock();
|
||||
|
||||
/// Need to clear type, because in description.init() the types are appended (emplace_back)
|
||||
description.types.clear();
|
||||
description.init(sample_block);
|
||||
|
||||
columns = description.sample_block.cloneEmptyColumns();
|
||||
const auto & storage_columns = storage_metadata->getColumns().getAllPhysical();
|
||||
auto insert_columns = std::make_shared<ASTExpressionList>();
|
||||
|
||||
auto table_id = storage->getStorageID();
|
||||
LOG_TRACE(&Poco::Logger::get("MaterializedPostgreSQLBuffer"), "New buffer for table {}.{} ({}), structure: {}",
|
||||
table_id.database_name, table_id.table_name, toString(table_id.uuid), sample_block.dumpStructure());
|
||||
|
||||
assert(description.sample_block.columns() == storage_columns.size());
|
||||
size_t idx = 0;
|
||||
|
||||
for (const auto & column : storage_columns)
|
||||
{
|
||||
if (description.types[idx].first == ExternalResultDescription::ValueType::vtArray)
|
||||
preparePostgreSQLArrayInfo(array_info, idx, description.sample_block.getByPosition(idx).type);
|
||||
idx++;
|
||||
|
||||
insert_columns->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
|
||||
}
|
||||
|
||||
columnsAST = std::move(insert_columns);
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::insertValue(Buffer & buffer, const std::string & value, size_t column_idx)
|
||||
{
|
||||
const auto & sample = buffer.description.sample_block.getByPosition(column_idx);
|
||||
bool is_nullable = buffer.description.types[column_idx].second;
|
||||
|
||||
if (is_nullable)
|
||||
{
|
||||
ColumnNullable & column_nullable = assert_cast<ColumnNullable &>(*buffer.columns[column_idx]);
|
||||
const auto & data_type = assert_cast<const DataTypeNullable &>(*sample.type);
|
||||
|
||||
insertPostgreSQLValue(
|
||||
column_nullable.getNestedColumn(), value,
|
||||
buffer.description.types[column_idx].first, data_type.getNestedType(), buffer.array_info, column_idx);
|
||||
|
||||
column_nullable.getNullMapData().emplace_back(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
insertPostgreSQLValue(
|
||||
*buffer.columns[column_idx], value,
|
||||
buffer.description.types[column_idx].first, sample.type,
|
||||
buffer.array_info, column_idx);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::insertDefaultValue(Buffer & buffer, size_t column_idx)
|
||||
{
|
||||
const auto & sample = buffer.description.sample_block.getByPosition(column_idx);
|
||||
insertDefaultPostgreSQLValue(*buffer.columns[column_idx], *sample.column);
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::readString(const char * message, size_t & pos, size_t size, String & result)
|
||||
{
|
||||
assert(size > pos + 2);
|
||||
char current = unhex2(message + pos);
|
||||
pos += 2;
|
||||
while (pos < size && current != '\0')
|
||||
{
|
||||
result += current;
|
||||
current = unhex2(message + pos);
|
||||
pos += 2;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
T MaterializedPostgreSQLConsumer::unhexN(const char * message, size_t pos, size_t n)
|
||||
{
|
||||
T result = 0;
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
{
|
||||
if (i) result <<= 8;
|
||||
result |= UInt32(unhex2(message + pos + 2 * i));
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
Int64 MaterializedPostgreSQLConsumer::readInt64(const char * message, size_t & pos, [[maybe_unused]] size_t size)
|
||||
{
|
||||
assert(size >= pos + 16);
|
||||
Int64 result = unhexN<Int64>(message, pos, 8);
|
||||
pos += 16;
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
Int32 MaterializedPostgreSQLConsumer::readInt32(const char * message, size_t & pos, [[maybe_unused]] size_t size)
|
||||
{
|
||||
assert(size >= pos + 8);
|
||||
Int32 result = unhexN<Int32>(message, pos, 4);
|
||||
pos += 8;
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
Int16 MaterializedPostgreSQLConsumer::readInt16(const char * message, size_t & pos, [[maybe_unused]] size_t size)
|
||||
{
|
||||
assert(size >= pos + 4);
|
||||
Int16 result = unhexN<Int16>(message, pos, 2);
|
||||
pos += 4;
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
Int8 MaterializedPostgreSQLConsumer::readInt8(const char * message, size_t & pos, [[maybe_unused]] size_t size)
|
||||
{
|
||||
assert(size >= pos + 2);
|
||||
Int8 result = unhex2(message + pos);
|
||||
pos += 2;
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::readTupleData(
|
||||
Buffer & buffer, const char * message, size_t & pos, [[maybe_unused]] size_t size, PostgreSQLQuery type, bool old_value)
|
||||
{
|
||||
Int16 num_columns = readInt16(message, pos, size);
|
||||
|
||||
auto proccess_column_value = [&](Int8 identifier, Int16 column_idx)
|
||||
{
|
||||
switch (identifier)
|
||||
{
|
||||
case 'n': /// NULL
|
||||
{
|
||||
insertDefaultValue(buffer, column_idx);
|
||||
break;
|
||||
}
|
||||
case 't': /// Text formatted value
|
||||
{
|
||||
Int32 col_len = readInt32(message, pos, size);
|
||||
String value;
|
||||
|
||||
for (Int32 i = 0; i < col_len; ++i)
|
||||
value += readInt8(message, pos, size);
|
||||
|
||||
insertValue(buffer, value, column_idx);
|
||||
break;
|
||||
}
|
||||
case 'u': /// TOAST value && unchanged at the same time. Actual value is not sent.
|
||||
{
|
||||
/// TOAST values are not supported. (TOAST values are values that are considered in postgres
|
||||
/// to be too large to be stored directly)
|
||||
LOG_WARNING(log, "Got TOAST value, which is not supported, default value will be used instead.");
|
||||
insertDefaultValue(buffer, column_idx);
|
||||
break;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
for (int column_idx = 0; column_idx < num_columns; ++column_idx)
|
||||
proccess_column_value(readInt8(message, pos, size), column_idx);
|
||||
|
||||
switch (type)
|
||||
{
|
||||
case PostgreSQLQuery::INSERT:
|
||||
{
|
||||
buffer.columns[num_columns]->insert(Int8(1));
|
||||
buffer.columns[num_columns + 1]->insert(lsn_value);
|
||||
|
||||
break;
|
||||
}
|
||||
case PostgreSQLQuery::DELETE:
|
||||
{
|
||||
buffer.columns[num_columns]->insert(Int8(-1));
|
||||
buffer.columns[num_columns + 1]->insert(lsn_value);
|
||||
|
||||
break;
|
||||
}
|
||||
case PostgreSQLQuery::UPDATE:
|
||||
{
|
||||
/// Process old value in case changed value is a primary key.
|
||||
if (old_value)
|
||||
buffer.columns[num_columns]->insert(Int8(-1));
|
||||
else
|
||||
buffer.columns[num_columns]->insert(Int8(1));
|
||||
|
||||
buffer.columns[num_columns + 1]->insert(lsn_value);
|
||||
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// https://www.postgresql.org/docs/13/protocol-logicalrep-message-formats.html
|
||||
void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * replication_message, size_t size)
|
||||
{
|
||||
/// Skip '\x'
|
||||
size_t pos = 2;
|
||||
char type = readInt8(replication_message, pos, size);
|
||||
// LOG_DEBUG(log, "Message type: {}, lsn string: {}, lsn value {}", type, current_lsn, lsn_value);
|
||||
|
||||
switch (type)
|
||||
{
|
||||
case 'B': // Begin
|
||||
{
|
||||
readInt64(replication_message, pos, size); /// Int64 transaction end lsn
|
||||
readInt64(replication_message, pos, size); /// Int64 transaction commit timestamp
|
||||
break;
|
||||
}
|
||||
case 'I': // Insert
|
||||
{
|
||||
Int32 relation_id = readInt32(replication_message, pos, size);
|
||||
|
||||
if (!isSyncAllowed(relation_id))
|
||||
return;
|
||||
|
||||
Int8 new_tuple = readInt8(replication_message, pos, size);
|
||||
const auto & table_name = relation_id_to_name[relation_id];
|
||||
auto buffer = buffers.find(table_name);
|
||||
assert(buffer != buffers.end());
|
||||
|
||||
if (new_tuple)
|
||||
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::INSERT);
|
||||
|
||||
break;
|
||||
}
|
||||
case 'U': // Update
|
||||
{
|
||||
Int32 relation_id = readInt32(replication_message, pos, size);
|
||||
|
||||
if (!isSyncAllowed(relation_id))
|
||||
return;
|
||||
|
||||
const auto & table_name = relation_id_to_name[relation_id];
|
||||
auto buffer = buffers.find(table_name);
|
||||
assert(buffer != buffers.end());
|
||||
|
||||
auto proccess_identifier = [&](Int8 identifier) -> bool
|
||||
{
|
||||
bool read_next = true;
|
||||
switch (identifier)
|
||||
{
|
||||
/// Only if changed column(s) are part of replica identity index (or primary keys if they are used instead).
|
||||
/// In this case, first comes a tuple with old replica identity indexes and all other values will come as
|
||||
/// nulls. Then comes a full new row.
|
||||
case 'K': [[fallthrough]];
|
||||
/// Old row. Only if replica identity is set to full. Does not really make sense to use it as
|
||||
/// it is much more efficient to use replica identity index, but support all possible cases.
|
||||
case 'O':
|
||||
{
|
||||
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE, true);
|
||||
break;
|
||||
}
|
||||
case 'N':
|
||||
{
|
||||
/// New row.
|
||||
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::UPDATE);
|
||||
read_next = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return read_next;
|
||||
};
|
||||
|
||||
/// Read either 'K' or 'O'. Never both of them. Also possible not to get both of them.
|
||||
bool read_next = proccess_identifier(readInt8(replication_message, pos, size));
|
||||
|
||||
/// 'N'. Always present, but could come in place of 'K' and 'O'.
|
||||
if (read_next)
|
||||
proccess_identifier(readInt8(replication_message, pos, size));
|
||||
|
||||
break;
|
||||
}
|
||||
case 'D': // Delete
|
||||
{
|
||||
Int32 relation_id = readInt32(replication_message, pos, size);
|
||||
|
||||
if (!isSyncAllowed(relation_id))
|
||||
return;
|
||||
|
||||
/// 0 or 1 if replica identity is set to full. For now only default replica identity is supported (with primary keys).
|
||||
readInt8(replication_message, pos, size);
|
||||
|
||||
const auto & table_name = relation_id_to_name[relation_id];
|
||||
auto buffer = buffers.find(table_name);
|
||||
assert(buffer != buffers.end());
|
||||
readTupleData(buffer->second, replication_message, pos, size, PostgreSQLQuery::DELETE);
|
||||
|
||||
break;
|
||||
}
|
||||
case 'C': // Commit
|
||||
{
|
||||
constexpr size_t unused_flags_len = 1;
|
||||
constexpr size_t commit_lsn_len = 8;
|
||||
constexpr size_t transaction_end_lsn_len = 8;
|
||||
constexpr size_t transaction_commit_timestamp_len = 8;
|
||||
pos += unused_flags_len + commit_lsn_len + transaction_end_lsn_len + transaction_commit_timestamp_len;
|
||||
|
||||
LOG_DEBUG(log, "Current lsn: {} = {}", current_lsn, getLSNValue(current_lsn)); /// Will be removed
|
||||
|
||||
final_lsn = current_lsn;
|
||||
break;
|
||||
}
|
||||
case 'R': // Relation
|
||||
{
|
||||
Int32 relation_id = readInt32(replication_message, pos, size);
|
||||
|
||||
String relation_namespace, relation_name;
|
||||
|
||||
readString(replication_message, pos, size, relation_namespace);
|
||||
readString(replication_message, pos, size, relation_name);
|
||||
|
||||
if (!isSyncAllowed(relation_id))
|
||||
return;
|
||||
|
||||
if (storages.find(relation_name) == storages.end())
|
||||
{
|
||||
markTableAsSkipped(relation_id, relation_name);
|
||||
LOG_ERROR(log,
|
||||
"Storage for table {} does not exist, but is included in replication stream. (Storages number: {})",
|
||||
relation_name, storages.size());
|
||||
return;
|
||||
}
|
||||
|
||||
assert(buffers.count(relation_name));
|
||||
|
||||
|
||||
/// 'd' - default (primary key if any)
|
||||
/// 'n' - nothing
|
||||
/// 'f' - all columns (set replica identity full)
|
||||
/// 'i' - user defined index with indisreplident set
|
||||
/// Only 'd' and 'i' - are supported.
|
||||
char replica_identity = readInt8(replication_message, pos, size);
|
||||
|
||||
if (replica_identity != 'd' && replica_identity != 'i')
|
||||
{
|
||||
LOG_WARNING(log,
|
||||
"Table has replica identity {} - not supported. A table must have a primary key or a replica identity index");
|
||||
markTableAsSkipped(relation_id, relation_name);
|
||||
return;
|
||||
}
|
||||
|
||||
Int16 num_columns = readInt16(replication_message, pos, size);
|
||||
|
||||
Int32 data_type_id;
|
||||
Int32 type_modifier; /// For example, n in varchar(n)
|
||||
|
||||
bool new_relation_definition = false;
|
||||
if (schema_data.find(relation_id) == schema_data.end())
|
||||
{
|
||||
relation_id_to_name[relation_id] = relation_name;
|
||||
schema_data.emplace(relation_id, SchemaData(num_columns));
|
||||
new_relation_definition = true;
|
||||
}
|
||||
|
||||
auto & current_schema_data = schema_data.find(relation_id)->second;
|
||||
|
||||
if (current_schema_data.number_of_columns != num_columns)
|
||||
{
|
||||
markTableAsSkipped(relation_id, relation_name);
|
||||
return;
|
||||
}
|
||||
|
||||
for (uint16_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
String column_name;
|
||||
readInt8(replication_message, pos, size); /// Marks column as part of replica identity index
|
||||
readString(replication_message, pos, size, column_name);
|
||||
|
||||
data_type_id = readInt32(replication_message, pos, size);
|
||||
type_modifier = readInt32(replication_message, pos, size);
|
||||
|
||||
if (new_relation_definition)
|
||||
{
|
||||
current_schema_data.column_identifiers.emplace_back(std::make_tuple(data_type_id, type_modifier));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (current_schema_data.column_identifiers[i].first != data_type_id
|
||||
|| current_schema_data.column_identifiers[i].second != type_modifier)
|
||||
{
|
||||
markTableAsSkipped(relation_id, relation_name);
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
tables_to_sync.insert(relation_name);
|
||||
|
||||
break;
|
||||
}
|
||||
case 'O': // Origin
|
||||
break;
|
||||
case 'Y': // Type
|
||||
break;
|
||||
case 'T': // Truncate
|
||||
break;
|
||||
default:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Unexpected byte1 value {} while parsing replication message", type);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::syncTables()
|
||||
{
|
||||
try
|
||||
{
|
||||
for (const auto & table_name : tables_to_sync)
|
||||
{
|
||||
auto & buffer = buffers.find(table_name)->second;
|
||||
Block result_rows = buffer.description.sample_block.cloneWithColumns(std::move(buffer.columns));
|
||||
|
||||
if (result_rows.rows())
|
||||
{
|
||||
auto storage = storages[table_name];
|
||||
|
||||
auto insert_context = Context::createCopy(context);
|
||||
insert_context->setInternalQuery(true);
|
||||
|
||||
auto insert = std::make_shared<ASTInsertQuery>();
|
||||
insert->table_id = storage->getStorageID();
|
||||
insert->columns = buffer.columnsAST;
|
||||
|
||||
InterpreterInsertQuery interpreter(insert, insert_context, true);
|
||||
auto block_io = interpreter.execute();
|
||||
OneBlockInputStream input(result_rows);
|
||||
|
||||
assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica table sync");
|
||||
copyData(input, *block_io.out);
|
||||
|
||||
buffer.columns = buffer.description.sample_block.cloneEmptyColumns();
|
||||
}
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn));
|
||||
|
||||
auto tx = std::make_shared<pqxx::nontransaction>(connection->getRef());
|
||||
current_lsn = advanceLSN(tx);
|
||||
tables_to_sync.clear();
|
||||
tx->commit();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
String MaterializedPostgreSQLConsumer::advanceLSN(std::shared_ptr<pqxx::nontransaction> tx)
|
||||
{
|
||||
std::string query_str = fmt::format("SELECT end_lsn FROM pg_replication_slot_advance('{}', '{}')", replication_slot_name, final_lsn);
|
||||
pqxx::result result{tx->exec(query_str)};
|
||||
|
||||
final_lsn = result[0][0].as<std::string>();
|
||||
LOG_TRACE(log, "Advanced LSN up to: {}", getLSNValue(final_lsn));
|
||||
return final_lsn;
|
||||
}
|
||||
|
||||
|
||||
/// Sync for some table might not be allowed if:
|
||||
/// 1. Table schema changed and might break synchronization.
|
||||
/// 2. There is no storage for this table. (As a result of some exception or incorrect pg_publication)
|
||||
bool MaterializedPostgreSQLConsumer::isSyncAllowed(Int32 relation_id)
|
||||
{
|
||||
auto table_with_lsn = skip_list.find(relation_id);
|
||||
|
||||
/// Table is not present in a skip list - allow synchronization.
|
||||
if (table_with_lsn == skip_list.end())
|
||||
return true;
|
||||
|
||||
const auto & table_start_lsn = table_with_lsn->second;
|
||||
|
||||
/// Table is in a skip list and has not yet received a valid lsn == it has not been reloaded.
|
||||
if (table_start_lsn.empty())
|
||||
return false;
|
||||
|
||||
/// Table has received a valid lsn, but it is not yet at a position, from which synchronization is
|
||||
/// allowed. It is allowed only after lsn position, returned with snapshot, from which
|
||||
/// table was reloaded.
|
||||
if (getLSNValue(current_lsn) >= getLSNValue(table_start_lsn))
|
||||
{
|
||||
LOG_TRACE(log, "Synchronization is resumed for table: {} (start_lsn: {})",
|
||||
relation_id_to_name[relation_id], table_start_lsn);
|
||||
|
||||
skip_list.erase(table_with_lsn);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const String & relation_name)
|
||||
{
|
||||
/// Empty lsn string means - continue waiting for valid lsn.
|
||||
skip_list.insert({relation_id, ""});
|
||||
|
||||
if (storages.count(relation_name))
|
||||
{
|
||||
/// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream
|
||||
/// and it receives first data after update.
|
||||
schema_data.erase(relation_id);
|
||||
|
||||
/// Clear table buffer.
|
||||
auto & buffer = buffers.find(relation_name)->second;
|
||||
buffer.columns = buffer.description.sample_block.cloneEmptyColumns();
|
||||
|
||||
if (allow_automatic_update)
|
||||
LOG_TRACE(log, "Table {} (relation_id: {}) is skipped temporarily. It will be reloaded in the background", relation_name, relation_id);
|
||||
else
|
||||
LOG_WARNING(log, "Table {} (relation_id: {}) is skipped, because table schema has changed", relation_name, relation_id);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// Read binary changes from replication slot via COPY command (starting from current lsn in a slot).
|
||||
bool MaterializedPostgreSQLConsumer::readFromReplicationSlot()
|
||||
{
|
||||
bool slot_empty = true;
|
||||
|
||||
try
|
||||
{
|
||||
auto tx = std::make_shared<pqxx::nontransaction>(connection->getRef());
|
||||
|
||||
/// Read up to max_block_size rows changes (upto_n_changes parameter). It might return larger number as the limit
|
||||
/// is checked only after each transaction block.
|
||||
/// Returns less than max_block_changes, if reached end of wal. Sync to table in this case.
|
||||
|
||||
std::string query_str = fmt::format(
|
||||
"select lsn, data FROM pg_logical_slot_peek_binary_changes("
|
||||
"'{}', NULL, {}, 'publication_names', '{}', 'proto_version', '1')",
|
||||
replication_slot_name, max_block_size, publication_name);
|
||||
|
||||
auto stream{pqxx::stream_from::query(*tx, query_str)};
|
||||
|
||||
while (true)
|
||||
{
|
||||
const std::vector<pqxx::zview> * row{stream.read_row()};
|
||||
|
||||
if (!row)
|
||||
{
|
||||
stream.complete();
|
||||
|
||||
if (slot_empty)
|
||||
{
|
||||
tx->commit();
|
||||
return false;
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
slot_empty = false;
|
||||
current_lsn = (*row)[0];
|
||||
lsn_value = getLSNValue(current_lsn);
|
||||
|
||||
processReplicationMessage((*row)[1].c_str(), (*row)[1].size());
|
||||
}
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
return false;
|
||||
}
|
||||
catch (const pqxx::broken_connection & e)
|
||||
{
|
||||
LOG_ERROR(log, "Connection error: {}", e.what());
|
||||
connection->tryUpdateConnection();
|
||||
return false;
|
||||
}
|
||||
catch (const pqxx::sql_error & e)
|
||||
{
|
||||
/// For now sql replication interface is used and it has the problem that it registers relcache
|
||||
/// callbacks on each pg_logical_slot_get_changes and there is no way to invalidate them:
|
||||
/// https://github.com/postgres/postgres/blob/master/src/backend/replication/pgoutput/pgoutput.c#L1128
|
||||
/// So at some point will get out of limit and then they will be cleaned.
|
||||
std::string error_message = e.what();
|
||||
if (error_message.find("out of relcache_callback_list slots") == std::string::npos)
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
|
||||
return false;
|
||||
}
|
||||
catch (const pqxx::conversion_error & e)
|
||||
{
|
||||
LOG_ERROR(log, "Conversion error: {}", e.what());
|
||||
return false;
|
||||
}
|
||||
catch (const pqxx::in_doubt_error & e)
|
||||
{
|
||||
LOG_ERROR(log, "PostgreSQL library has some doubts: {}", e.what());
|
||||
return false;
|
||||
}
|
||||
catch (const pqxx::internal_error & e)
|
||||
{
|
||||
LOG_ERROR(log, "PostgreSQL library internal error: {}", e.what());
|
||||
return false;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Since reading is done from a background task, it is important to catch any possible error
|
||||
/// in order to understand why something does not work.
|
||||
try
|
||||
{
|
||||
std::rethrow_exception(std::current_exception());
|
||||
}
|
||||
catch (const std::exception& e)
|
||||
{
|
||||
LOG_ERROR(log, "Unexpected error: {}", e.what());
|
||||
}
|
||||
}
|
||||
|
||||
if (!tables_to_sync.empty())
|
||||
syncTables();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool MaterializedPostgreSQLConsumer::consume(std::vector<std::pair<Int32, String>> & skipped_tables)
|
||||
{
|
||||
/// Check if there are tables, which are skipped from being updated by changes from replication stream,
|
||||
/// because schema changes were detected. Update them, if it is allowed.
|
||||
if (allow_automatic_update && !skip_list.empty())
|
||||
{
|
||||
for (const auto & [relation_id, lsn] : skip_list)
|
||||
{
|
||||
/// Non-empty lsn in this place means that table was already updated, but no changes for that table were
|
||||
/// received in a previous stream. A table is removed from skip list only when there came
|
||||
/// changes for table with lsn higher than lsn of snapshot, from which table was reloaded. Since table
|
||||
/// reaload and reading from replication stream are done in the same thread, no lsn will be skipped
|
||||
/// between these two events.
|
||||
if (lsn.empty())
|
||||
skipped_tables.emplace_back(std::make_pair(relation_id, relation_id_to_name[relation_id]));
|
||||
}
|
||||
}
|
||||
|
||||
/// Read up to max_block_size changed (approximately - in same cases might be more).
|
||||
/// false: no data was read, reschedule.
|
||||
/// true: some data was read, schedule as soon as possible.
|
||||
return readFromReplicationSlot();
|
||||
}
|
||||
|
||||
|
||||
void MaterializedPostgreSQLConsumer::updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn)
|
||||
{
|
||||
/// Cache new pointer to replacingMergeTree table.
|
||||
storages[table_name] = nested_storage;
|
||||
|
||||
/// Create a new empty buffer (with updated metadata), where data is first loaded before syncing into actual table.
|
||||
auto & buffer = buffers.find(table_name)->second;
|
||||
buffer.createEmptyBuffer(nested_storage);
|
||||
|
||||
/// Set start position to valid lsn. Before it was an empty string. Further read for table allowed, if it has a valid lsn.
|
||||
skip_list[table_id] = table_start_lsn;
|
||||
}
|
||||
|
||||
}
|
146
src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h
Normal file
146
src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h
Normal file
@ -0,0 +1,146 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/PostgreSQL/Connection.h>
|
||||
#include <Core/PostgreSQL/insertPostgreSQLValue.h>
|
||||
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Core/Names.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MaterializedPostgreSQLConsumer
|
||||
{
|
||||
public:
|
||||
using Storages = std::unordered_map<String, StoragePtr>;
|
||||
|
||||
MaterializedPostgreSQLConsumer(
|
||||
ContextPtr context_,
|
||||
std::shared_ptr<postgres::Connection> connection_,
|
||||
const String & replication_slot_name_,
|
||||
const String & publication_name_,
|
||||
const String & start_lsn,
|
||||
const size_t max_block_size_,
|
||||
bool allow_automatic_update_,
|
||||
Storages storages_);
|
||||
|
||||
bool consume(std::vector<std::pair<Int32, String>> & skipped_tables);
|
||||
|
||||
/// Called from reloadFromSnapshot by replication handler. This method is needed to move a table back into synchronization
|
||||
/// process if it was skipped due to schema changes.
|
||||
void updateNested(const String & table_name, StoragePtr nested_storage, Int32 table_id, const String & table_start_lsn);
|
||||
|
||||
private:
|
||||
/// Read approximarely up to max_block_size changes from WAL.
|
||||
bool readFromReplicationSlot();
|
||||
|
||||
void syncTables();
|
||||
|
||||
String advanceLSN(std::shared_ptr<pqxx::nontransaction> ntx);
|
||||
|
||||
void processReplicationMessage(const char * replication_message, size_t size);
|
||||
|
||||
bool isSyncAllowed(Int32 relation_id);
|
||||
|
||||
struct Buffer
|
||||
{
|
||||
ExternalResultDescription description;
|
||||
MutableColumns columns;
|
||||
|
||||
/// Needed to pass to insert query columns list in syncTables().
|
||||
std::shared_ptr<ASTExpressionList> columnsAST;
|
||||
|
||||
/// Needed for insertPostgreSQLValue() method to parse array
|
||||
std::unordered_map<size_t, PostgreSQLArrayInfo> array_info;
|
||||
|
||||
Buffer(StoragePtr storage) { createEmptyBuffer(storage); }
|
||||
void createEmptyBuffer(StoragePtr storage);
|
||||
};
|
||||
|
||||
using Buffers = std::unordered_map<String, Buffer>;
|
||||
|
||||
static void insertDefaultValue(Buffer & buffer, size_t column_idx);
|
||||
static void insertValue(Buffer & buffer, const std::string & value, size_t column_idx);
|
||||
|
||||
enum class PostgreSQLQuery
|
||||
{
|
||||
INSERT,
|
||||
UPDATE,
|
||||
DELETE
|
||||
};
|
||||
|
||||
void readTupleData(Buffer & buffer, const char * message, size_t & pos, size_t size, PostgreSQLQuery type, bool old_value = false);
|
||||
|
||||
template<typename T>
|
||||
static T unhexN(const char * message, size_t pos, size_t n);
|
||||
static void readString(const char * message, size_t & pos, size_t size, String & result);
|
||||
static Int64 readInt64(const char * message, size_t & pos, size_t size);
|
||||
static Int32 readInt32(const char * message, size_t & pos, size_t size);
|
||||
static Int16 readInt16(const char * message, size_t & pos, size_t size);
|
||||
static Int8 readInt8(const char * message, size_t & pos, size_t size);
|
||||
|
||||
void markTableAsSkipped(Int32 relation_id, const String & relation_name);
|
||||
|
||||
/// lsn - log sequnce nuumber, like wal offset (64 bit).
|
||||
Int64 getLSNValue(const std::string & lsn)
|
||||
{
|
||||
UInt32 upper_half, lower_half;
|
||||
std::sscanf(lsn.data(), "%X/%X", &upper_half, &lower_half);
|
||||
return (static_cast<Int64>(upper_half) << 32) + lower_half;
|
||||
}
|
||||
|
||||
Poco::Logger * log;
|
||||
ContextPtr context;
|
||||
const std::string replication_slot_name, publication_name;
|
||||
|
||||
std::shared_ptr<postgres::Connection> connection;
|
||||
|
||||
std::string current_lsn, final_lsn;
|
||||
|
||||
/// current_lsn converted from String to Int64 via getLSNValue().
|
||||
UInt64 lsn_value;
|
||||
|
||||
const size_t max_block_size;
|
||||
bool allow_automatic_update;
|
||||
|
||||
String table_to_insert;
|
||||
|
||||
/// List of tables which need to be synced after last replication stream.
|
||||
std::unordered_set<std::string> tables_to_sync;
|
||||
|
||||
Storages storages;
|
||||
Buffers buffers;
|
||||
|
||||
std::unordered_map<Int32, String> relation_id_to_name;
|
||||
|
||||
struct SchemaData
|
||||
{
|
||||
Int16 number_of_columns;
|
||||
/// data_type_id and type_modifier
|
||||
std::vector<std::pair<Int32, Int32>> column_identifiers;
|
||||
|
||||
SchemaData(Int16 number_of_columns_) : number_of_columns(number_of_columns_) {}
|
||||
};
|
||||
|
||||
/// Cache for table schema data to be able to detect schema changes, because ddl is not
|
||||
/// replicated with postgresql logical replication protocol, but some table schema info
|
||||
/// is received if it is the first time we received dml message for given relation in current session or
|
||||
/// if relation definition has changed since the last relation definition message.
|
||||
std::unordered_map<Int32, SchemaData> schema_data;
|
||||
|
||||
/// skip_list contains relation ids for tables on which ddl was performed, which can break synchronization.
|
||||
/// This breaking changes are detected in replication stream in according replication message and table is added to skip list.
|
||||
/// After it is finished, a temporary replication slot is created with 'export snapshot' option, and start_lsn is returned.
|
||||
/// Skipped tables are reloaded from snapshot (nested tables are also updated). Afterwards, if a replication message is
|
||||
/// related to a table in a skip_list, we compare current lsn with start_lsn, which was returned with according snapshot.
|
||||
/// If current_lsn >= table_start_lsn, we can safely remove table from skip list and continue its synchronization.
|
||||
/// No needed message, related to reloaded table will be missed, because messages are not consumed in the meantime,
|
||||
/// i.e. we will not miss the first start_lsn position for reloaded table.
|
||||
std::unordered_map<Int32, String> skip_list;
|
||||
};
|
||||
}
|
45
src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp
Normal file
45
src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.cpp
Normal file
@ -0,0 +1,45 @@
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_SETTING;
|
||||
}
|
||||
|
||||
IMPLEMENT_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
|
||||
void MaterializedPostgreSQLSettings::loadFromQuery(ASTStorage & storage_def)
|
||||
{
|
||||
if (storage_def.settings)
|
||||
{
|
||||
try
|
||||
{
|
||||
applyChanges(storage_def.settings->changes);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::UNKNOWN_SETTING)
|
||||
e.addMessage("for storage " + storage_def.engine->name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto settings_ast = std::make_shared<ASTSetQuery>();
|
||||
settings_ast->is_standalone = false;
|
||||
storage_def.set(storage_def.settings, settings_ast);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
30
src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h
Normal file
30
src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h
Normal file
@ -0,0 +1,30 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Core/BaseSettings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTStorage;
|
||||
|
||||
|
||||
#define LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(M) \
|
||||
M(UInt64, materialized_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \
|
||||
M(String, materialized_postgresql_tables_list, "", "List of tables for MaterializedPostgreSQL database engine", 0) \
|
||||
M(Bool, materialized_postgresql_allow_automatic_update, 0, "Allow to reload table in the background, when schema changes are detected", 0) \
|
||||
|
||||
DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS)
|
||||
|
||||
struct MaterializedPostgreSQLSettings : public BaseSettings<MaterializedPostgreSQLSettingsTraits>
|
||||
{
|
||||
void loadFromQuery(ASTStorage & storage_def);
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
629
src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp
Normal file
629
src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp
Normal file
@ -0,0 +1,629 @@
|
||||
#include "PostgreSQLReplicationHandler.h"
|
||||
|
||||
#include <DataStreams/PostgreSQLBlockInputStream.h>
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/InterpreterInsertQuery.h>
|
||||
#include <Interpreters/InterpreterRenameQuery.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static const auto RESCHEDULE_MS = 500;
|
||||
static const auto BACKOFF_TRESHOLD_MS = 10000;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
PostgreSQLReplicationHandler::PostgreSQLReplicationHandler(
|
||||
const String & replication_identifier,
|
||||
const String & remote_database_name_,
|
||||
const String & current_database_name_,
|
||||
const postgres::ConnectionInfo & connection_info_,
|
||||
ContextPtr context_,
|
||||
const size_t max_block_size_,
|
||||
bool allow_automatic_update_,
|
||||
bool is_materialized_postgresql_database_,
|
||||
const String tables_list_)
|
||||
: log(&Poco::Logger::get("PostgreSQLReplicationHandler"))
|
||||
, context(context_)
|
||||
, remote_database_name(remote_database_name_)
|
||||
, current_database_name(current_database_name_)
|
||||
, connection_info(connection_info_)
|
||||
, max_block_size(max_block_size_)
|
||||
, allow_automatic_update(allow_automatic_update_)
|
||||
, is_materialized_postgresql_database(is_materialized_postgresql_database_)
|
||||
, tables_list(tables_list_)
|
||||
, connection(std::make_shared<postgres::Connection>(connection_info_))
|
||||
, milliseconds_to_wait(RESCHEDULE_MS)
|
||||
{
|
||||
replication_slot = fmt::format("{}_ch_replication_slot", replication_identifier);
|
||||
publication_name = fmt::format("{}_ch_publication", replication_identifier);
|
||||
|
||||
startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); });
|
||||
consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); });
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, StorageMaterializedPostgreSQL * storage)
|
||||
{
|
||||
materialized_storages[table_name] = storage;
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::startup()
|
||||
{
|
||||
startup_task->activateAndSchedule();
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::waitConnectionAndStart()
|
||||
{
|
||||
try
|
||||
{
|
||||
connection->connect(); /// Will throw pqxx::broken_connection if no connection at the moment
|
||||
startSynchronization(false);
|
||||
}
|
||||
catch (const pqxx::broken_connection & pqxx_error)
|
||||
{
|
||||
LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", pqxx_error.what());
|
||||
startup_task->scheduleAfter(RESCHEDULE_MS);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::shutdown()
|
||||
{
|
||||
stop_synchronization.store(true);
|
||||
startup_task->deactivate();
|
||||
consumer_task->deactivate();
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error)
|
||||
{
|
||||
{
|
||||
pqxx::work tx(connection->getRef());
|
||||
createPublicationIfNeeded(tx);
|
||||
tx.commit();
|
||||
}
|
||||
|
||||
postgres::Connection replication_connection(connection_info, /* replication */true);
|
||||
pqxx::nontransaction tx(replication_connection.getRef());
|
||||
|
||||
/// List of nested tables (table_name -> nested_storage), which is passed to replication consumer.
|
||||
std::unordered_map<String, StoragePtr> nested_storages;
|
||||
|
||||
/// snapshot_name is initialized only if a new replication slot is created.
|
||||
/// start_lsn is initialized in two places:
|
||||
/// 1. if replication slot does not exist, start_lsn will be returned with its creation return parameters;
|
||||
/// 2. if replication slot already exist, start_lsn is read from pg_replication_slots as
|
||||
/// `confirmed_flush_lsn` - the address (LSN) up to which the logical slot's consumer has confirmed receiving data.
|
||||
/// Data older than this is not available anymore.
|
||||
/// TODO: more tests
|
||||
String snapshot_name, start_lsn;
|
||||
|
||||
auto initial_sync = [&]()
|
||||
{
|
||||
createReplicationSlot(tx, start_lsn, snapshot_name);
|
||||
|
||||
for (const auto & [table_name, storage] : materialized_storages)
|
||||
{
|
||||
try
|
||||
{
|
||||
nested_storages[table_name] = loadFromSnapshot(snapshot_name, table_name, storage->as <StorageMaterializedPostgreSQL>());
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while loading table {}.{}", remote_database_name, table_name);
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
|
||||
/// Throw in case of single MaterializedPostgreSQL storage, because initial setup is done immediately
|
||||
/// (unlike database engine where it is done in a separate thread).
|
||||
if (throw_on_error)
|
||||
throw;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
/// There is one replication slot for each replication handler. In case of MaterializedPostgreSQL database engine,
|
||||
/// there is one replication slot per database. Its lifetime must be equal to the lifetime of replication handler.
|
||||
/// Recreation of a replication slot imposes reloading of all tables.
|
||||
if (!isReplicationSlotExist(tx, start_lsn, /* temporary */false))
|
||||
{
|
||||
initial_sync();
|
||||
}
|
||||
/// Replication slot depends on publication, so if replication slot exists and new
|
||||
/// publication was just created - drop that replication slot and start from scratch.
|
||||
/// TODO: tests
|
||||
else if (new_publication_created)
|
||||
{
|
||||
dropReplicationSlot(tx);
|
||||
initial_sync();
|
||||
}
|
||||
/// Synchronization and initial load already took place - do not create any new tables, just fetch StoragePtr's
|
||||
/// and pass them to replication consumer.
|
||||
else
|
||||
{
|
||||
for (const auto & [table_name, storage] : materialized_storages)
|
||||
{
|
||||
auto * materialized_storage = storage->as <StorageMaterializedPostgreSQL>();
|
||||
try
|
||||
{
|
||||
/// Try load nested table, set materialized table metadata.
|
||||
nested_storages[table_name] = materialized_storage->prepare();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while loading table {}.{}", remote_database_name, table_name);
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
|
||||
if (throw_on_error)
|
||||
throw;
|
||||
}
|
||||
}
|
||||
LOG_TRACE(log, "Loaded {} tables", nested_storages.size());
|
||||
}
|
||||
|
||||
tx.commit();
|
||||
|
||||
/// Pass current connection to consumer. It is not std::moved implicitly, but a shared_ptr is passed.
|
||||
/// Consumer and replication handler are always executed one after another (not concurrently) and share the same connection.
|
||||
/// (Apart from the case, when shutdownFinal is called).
|
||||
/// Handler uses it only for loadFromSnapshot and shutdown methods.
|
||||
consumer = std::make_shared<MaterializedPostgreSQLConsumer>(
|
||||
context,
|
||||
connection,
|
||||
replication_slot,
|
||||
publication_name,
|
||||
start_lsn,
|
||||
max_block_size,
|
||||
allow_automatic_update,
|
||||
nested_storages);
|
||||
|
||||
consumer_task->activateAndSchedule();
|
||||
|
||||
/// Do not rely anymore on saved storage pointers.
|
||||
materialized_storages.clear();
|
||||
}
|
||||
|
||||
|
||||
StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name, const String & table_name,
|
||||
StorageMaterializedPostgreSQL * materialized_storage)
|
||||
{
|
||||
auto tx = std::make_shared<pqxx::ReplicationTransaction>(connection->getRef());
|
||||
|
||||
std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name);
|
||||
tx->exec(query_str);
|
||||
|
||||
/// Load from snapshot, which will show table state before creation of replication slot.
|
||||
/// Already connected to needed database, no need to add it to query.
|
||||
query_str = fmt::format("SELECT * FROM {}", table_name);
|
||||
|
||||
materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name));
|
||||
auto nested_storage = materialized_storage->getNested();
|
||||
|
||||
auto insert = std::make_shared<ASTInsertQuery>();
|
||||
insert->table_id = nested_storage->getStorageID();
|
||||
|
||||
auto insert_context = materialized_storage->getNestedTableContext();
|
||||
|
||||
InterpreterInsertQuery interpreter(insert, insert_context);
|
||||
auto block_io = interpreter.execute();
|
||||
|
||||
const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata();
|
||||
auto sample_block = storage_metadata.getSampleBlockNonMaterialized();
|
||||
|
||||
PostgreSQLTransactionBlockInputStream<pqxx::ReplicationTransaction> input(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE);
|
||||
assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot");
|
||||
copyData(input, *block_io.out);
|
||||
|
||||
nested_storage = materialized_storage->prepare();
|
||||
auto nested_table_id = nested_storage->getStorageID();
|
||||
LOG_TRACE(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid));
|
||||
|
||||
return nested_storage;
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::consumerFunc()
|
||||
{
|
||||
std::vector<std::pair<Int32, String>> skipped_tables;
|
||||
|
||||
bool schedule_now = consumer->consume(skipped_tables);
|
||||
|
||||
if (!skipped_tables.empty())
|
||||
{
|
||||
try
|
||||
{
|
||||
reloadFromSnapshot(skipped_tables);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
if (stop_synchronization)
|
||||
{
|
||||
LOG_TRACE(log, "Replication thread is stopped");
|
||||
return;
|
||||
}
|
||||
|
||||
if (schedule_now)
|
||||
{
|
||||
milliseconds_to_wait = RESCHEDULE_MS;
|
||||
consumer_task->schedule();
|
||||
|
||||
LOG_DEBUG(log, "Scheduling replication thread: now");
|
||||
}
|
||||
else
|
||||
{
|
||||
consumer_task->scheduleAfter(milliseconds_to_wait);
|
||||
if (milliseconds_to_wait < BACKOFF_TRESHOLD_MS)
|
||||
milliseconds_to_wait *= 2;
|
||||
|
||||
LOG_TRACE(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool PostgreSQLReplicationHandler::isPublicationExist(pqxx::work & tx)
|
||||
{
|
||||
std::string query_str = fmt::format("SELECT exists (SELECT 1 FROM pg_publication WHERE pubname = '{}')", publication_name);
|
||||
pqxx::result result{tx.exec(query_str)};
|
||||
assert(!result.empty());
|
||||
bool publication_exists = (result[0][0].as<std::string>() == "t");
|
||||
|
||||
if (publication_exists)
|
||||
LOG_INFO(log, "Publication {} already exists. Using existing version", publication_name);
|
||||
|
||||
return publication_exists;
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx, bool create_without_check)
|
||||
{
|
||||
/// For database engine a publication can be created earlier than in startReplication().
|
||||
if (new_publication_created)
|
||||
return;
|
||||
|
||||
if (create_without_check || !isPublicationExist(tx))
|
||||
{
|
||||
if (tables_list.empty())
|
||||
{
|
||||
for (const auto & storage_data : materialized_storages)
|
||||
{
|
||||
if (!tables_list.empty())
|
||||
tables_list += ", ";
|
||||
tables_list += storage_data.first;
|
||||
}
|
||||
}
|
||||
|
||||
if (tables_list.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No table found to be replicated");
|
||||
|
||||
/// 'ONLY' means just a table, without descendants.
|
||||
std::string query_str = fmt::format("CREATE PUBLICATION {} FOR TABLE ONLY {}", publication_name, tables_list);
|
||||
try
|
||||
{
|
||||
tx.exec(query_str);
|
||||
new_publication_created = true;
|
||||
LOG_TRACE(log, "Created publication {} with tables list: {}", publication_name, tables_list);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while creating pg_publication");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & tx, String & start_lsn, bool temporary)
|
||||
{
|
||||
String slot_name;
|
||||
if (temporary)
|
||||
slot_name = replication_slot + "_tmp";
|
||||
else
|
||||
slot_name = replication_slot;
|
||||
|
||||
String query_str = fmt::format("SELECT active, restart_lsn, confirmed_flush_lsn FROM pg_replication_slots WHERE slot_name = '{}'", slot_name);
|
||||
pqxx::result result{tx.exec(query_str)};
|
||||
|
||||
/// Replication slot does not exist
|
||||
if (result.empty())
|
||||
return false;
|
||||
|
||||
start_lsn = result[0][2].as<std::string>();
|
||||
|
||||
LOG_TRACE(log, "Replication slot {} already exists (active: {}). Restart lsn position: {}, confirmed flush lsn: {}",
|
||||
slot_name, result[0][0].as<bool>(), result[0][1].as<std::string>(), start_lsn);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::createReplicationSlot(
|
||||
pqxx::nontransaction & tx, String & start_lsn, String & snapshot_name, bool temporary)
|
||||
{
|
||||
String query_str, slot_name;
|
||||
if (temporary)
|
||||
slot_name = replication_slot + "_tmp";
|
||||
else
|
||||
slot_name = replication_slot;
|
||||
|
||||
query_str = fmt::format("CREATE_REPLICATION_SLOT {} LOGICAL pgoutput EXPORT_SNAPSHOT", slot_name);
|
||||
|
||||
try
|
||||
{
|
||||
pqxx::result result{tx.exec(query_str)};
|
||||
start_lsn = result[0][1].as<std::string>();
|
||||
snapshot_name = result[0][2].as<std::string>();
|
||||
LOG_TRACE(log, "Created replication slot: {}, start lsn: {}", replication_slot, start_lsn);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while creating PostgreSQL replication slot {}", slot_name);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx, bool temporary)
|
||||
{
|
||||
std::string slot_name;
|
||||
if (temporary)
|
||||
slot_name = replication_slot + "_tmp";
|
||||
else
|
||||
slot_name = replication_slot;
|
||||
|
||||
std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name);
|
||||
|
||||
tx.exec(query_str);
|
||||
LOG_TRACE(log, "Dropped replication slot: {}", slot_name);
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx)
|
||||
{
|
||||
std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name);
|
||||
tx.exec(query_str);
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::shutdownFinal()
|
||||
{
|
||||
try
|
||||
{
|
||||
shutdown();
|
||||
|
||||
connection->execWithRetry([&](pqxx::nontransaction & tx){ dropPublication(tx); });
|
||||
String last_committed_lsn;
|
||||
|
||||
connection->execWithRetry([&](pqxx::nontransaction & tx)
|
||||
{
|
||||
if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */false))
|
||||
dropReplicationSlot(tx, /* temporary */false);
|
||||
});
|
||||
|
||||
connection->execWithRetry([&](pqxx::nontransaction & tx)
|
||||
{
|
||||
if (isReplicationSlotExist(tx, last_committed_lsn, /* temporary */true))
|
||||
dropReplicationSlot(tx, /* temporary */true);
|
||||
});
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while dropping replication slot: {}", replication_slot);
|
||||
LOG_ERROR(log, "Failed to drop replication slot: {}. It must be dropped manually.", replication_slot);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// Used by MaterializedPostgreSQL database engine.
|
||||
NameSet PostgreSQLReplicationHandler::fetchRequiredTables(postgres::Connection & connection_)
|
||||
{
|
||||
pqxx::work tx(connection_.getRef());
|
||||
bool publication_exists_before_startup = isPublicationExist(tx);
|
||||
NameSet result_tables;
|
||||
|
||||
Strings expected_tables;
|
||||
if (!tables_list.empty())
|
||||
{
|
||||
splitInto<','>(expected_tables, tables_list);
|
||||
if (expected_tables.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse tables list: {}", tables_list);
|
||||
for (auto & table_name : expected_tables)
|
||||
boost::trim(table_name);
|
||||
}
|
||||
|
||||
if (publication_exists_before_startup)
|
||||
{
|
||||
if (tables_list.empty())
|
||||
{
|
||||
/// There is no tables list, but publication already exists, then the expected behaviour
|
||||
/// is to replicate the whole database. But it could be a server restart, so we can't drop it.
|
||||
LOG_WARNING(log,
|
||||
"Publication {} already exists and tables list is empty. Assuming publication is correct",
|
||||
publication_name);
|
||||
|
||||
result_tables = fetchPostgreSQLTablesList(tx);
|
||||
}
|
||||
/// Check tables list from publication is the same as expected tables list.
|
||||
/// If not - drop publication and return expected tables list.
|
||||
else
|
||||
{
|
||||
result_tables = fetchTablesFromPublication(tx);
|
||||
NameSet diff;
|
||||
std::set_symmetric_difference(expected_tables.begin(), expected_tables.end(),
|
||||
result_tables.begin(), result_tables.end(),
|
||||
std::inserter(diff, diff.begin()));
|
||||
if (!diff.empty())
|
||||
{
|
||||
String diff_tables;
|
||||
for (const auto & table_name : diff)
|
||||
{
|
||||
if (!diff_tables.empty())
|
||||
diff_tables += ", ";
|
||||
diff_tables += table_name;
|
||||
}
|
||||
|
||||
LOG_WARNING(log,
|
||||
"Publication {} already exists, but specified tables list differs from publication tables list in tables: {}",
|
||||
publication_name, diff_tables);
|
||||
|
||||
connection->execWithRetry([&](pqxx::nontransaction & tx_){ dropPublication(tx_); });
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!tables_list.empty())
|
||||
{
|
||||
tx.commit();
|
||||
return NameSet(expected_tables.begin(), expected_tables.end());
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Fetch all tables list from database. Publication does not exist yet, which means
|
||||
/// that no replication took place. Publication will be created in
|
||||
/// startSynchronization method.
|
||||
result_tables = fetchPostgreSQLTablesList(tx);
|
||||
}
|
||||
}
|
||||
|
||||
tx.commit();
|
||||
return result_tables;
|
||||
}
|
||||
|
||||
|
||||
NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx)
|
||||
{
|
||||
std::string query = fmt::format("SELECT tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name);
|
||||
std::unordered_set<std::string> tables;
|
||||
|
||||
for (auto table_name : tx.stream<std::string>(query))
|
||||
tables.insert(std::get<0>(table_name));
|
||||
|
||||
return tables;
|
||||
}
|
||||
|
||||
|
||||
PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure(
|
||||
pqxx::ReplicationTransaction & tx, const std::string & table_name) const
|
||||
{
|
||||
if (!is_materialized_postgresql_database)
|
||||
return nullptr;
|
||||
|
||||
return std::make_unique<PostgreSQLTableStructure>(fetchPostgreSQLTableStructure(tx, table_name, true, true, true));
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vector<std::pair<Int32, String>> & relation_data)
|
||||
{
|
||||
/// If table schema has changed, the table stops consuming changes from replication stream.
|
||||
/// If `allow_automatic_update` is true, create a new table in the background, load new table schema
|
||||
/// and all data from scratch. Then execute REPLACE query.
|
||||
/// This is only allowed for MaterializedPostgreSQL database engine.
|
||||
try
|
||||
{
|
||||
postgres::Connection replication_connection(connection_info, /* replication */true);
|
||||
pqxx::nontransaction tx(replication_connection.getRef());
|
||||
|
||||
String snapshot_name, start_lsn;
|
||||
|
||||
if (isReplicationSlotExist(tx, start_lsn, /* temporary */true))
|
||||
dropReplicationSlot(tx, /* temporary */true);
|
||||
|
||||
createReplicationSlot(tx, start_lsn, snapshot_name, /* temporary */true);
|
||||
|
||||
for (const auto & [relation_id, table_name] : relation_data)
|
||||
{
|
||||
auto storage = DatabaseCatalog::instance().getTable(StorageID(current_database_name, table_name), context);
|
||||
auto * materialized_storage = storage->as <StorageMaterializedPostgreSQL>();
|
||||
|
||||
/// If for some reason this temporary table already exists - also drop it.
|
||||
auto temp_materialized_storage = materialized_storage->createTemporary();
|
||||
|
||||
/// This snapshot is valid up to the end of the transaction, which exported it.
|
||||
StoragePtr temp_nested_storage = loadFromSnapshot(snapshot_name, table_name,
|
||||
temp_materialized_storage->as <StorageMaterializedPostgreSQL>());
|
||||
|
||||
auto table_id = materialized_storage->getNestedStorageID();
|
||||
auto temp_table_id = temp_nested_storage->getStorageID();
|
||||
|
||||
LOG_TRACE(log, "Starting background update of table {} with table {}",
|
||||
table_id.getNameForLogs(), temp_table_id.getNameForLogs());
|
||||
|
||||
auto ast_rename = std::make_shared<ASTRenameQuery>();
|
||||
ASTRenameQuery::Element elem
|
||||
{
|
||||
ASTRenameQuery::Table{table_id.database_name, table_id.table_name},
|
||||
ASTRenameQuery::Table{temp_table_id.database_name, temp_table_id.table_name}
|
||||
};
|
||||
ast_rename->elements.push_back(std::move(elem));
|
||||
ast_rename->exchange = true;
|
||||
|
||||
auto nested_context = materialized_storage->getNestedTableContext();
|
||||
|
||||
try
|
||||
{
|
||||
auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout);
|
||||
InterpreterRenameQuery(ast_rename, nested_context).execute();
|
||||
|
||||
{
|
||||
auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name),
|
||||
nested_context);
|
||||
auto nested_table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout);
|
||||
auto nested_table_id = nested_storage->getStorageID();
|
||||
|
||||
materialized_storage->setNestedStorageID(nested_table_id);
|
||||
nested_storage = materialized_storage->prepare();
|
||||
|
||||
auto nested_storage_metadata = nested_storage->getInMemoryMetadataPtr();
|
||||
auto nested_sample_block = nested_storage_metadata->getSampleBlock();
|
||||
LOG_TRACE(log, "Updated table {}. New structure: {}",
|
||||
nested_table_id.getNameForLogs(), nested_sample_block.dumpStructure());
|
||||
|
||||
auto materialized_storage_metadata = nested_storage->getInMemoryMetadataPtr();
|
||||
auto materialized_sample_block = materialized_storage_metadata->getSampleBlock();
|
||||
|
||||
assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background");
|
||||
|
||||
/// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position.
|
||||
consumer->updateNested(table_name, nested_storage, relation_id, start_lsn);
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Dropping table {}", temp_table_id.getNameForLogs());
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, temp_table_id, true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
dropReplicationSlot(tx, /* temporary */true);
|
||||
tx.commit();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
127
src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h
Normal file
127
src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h
Normal file
@ -0,0 +1,127 @@
|
||||
#pragma once
|
||||
|
||||
#include "MaterializedPostgreSQLConsumer.h"
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Core/PostgreSQL/Utils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// IDEA: There is ALTER PUBLICATION command to dynamically add and remove tables for replicating (the command is transactional).
|
||||
/// (Probably, if in a replication stream comes a relation name, which does not currently
|
||||
/// exist in CH, it can be loaded via snapshot while stream is stopped and then comparing wal positions with
|
||||
/// current lsn and table start lsn.
|
||||
|
||||
class StorageMaterializedPostgreSQL;
|
||||
|
||||
class PostgreSQLReplicationHandler
|
||||
{
|
||||
public:
|
||||
PostgreSQLReplicationHandler(
|
||||
const String & replication_identifier,
|
||||
const String & remote_database_name_,
|
||||
const String & current_database_name_,
|
||||
const postgres::ConnectionInfo & connection_info_,
|
||||
ContextPtr context_,
|
||||
const size_t max_block_size_,
|
||||
bool allow_automatic_update_,
|
||||
bool is_materialized_postgresql_database_,
|
||||
const String tables_list = "");
|
||||
|
||||
/// Activate task to be run from a separate thread: wait until connection is available and call startReplication().
|
||||
void startup();
|
||||
|
||||
/// Stop replication without cleanup.
|
||||
void shutdown();
|
||||
|
||||
/// Clean up replication: remove publication and replication slots.
|
||||
void shutdownFinal();
|
||||
|
||||
/// Add storage pointer to let handler know which tables it needs to keep in sync.
|
||||
void addStorage(const std::string & table_name, StorageMaterializedPostgreSQL * storage);
|
||||
|
||||
/// Fetch list of tables which are going to be replicated. Used for database engine.
|
||||
NameSet fetchRequiredTables(postgres::Connection & connection_);
|
||||
|
||||
/// Start replication setup immediately.
|
||||
void startSynchronization(bool throw_on_error);
|
||||
|
||||
private:
|
||||
using MaterializedStorages = std::unordered_map<String, StorageMaterializedPostgreSQL *>;
|
||||
|
||||
/// Methods to manage Publication.
|
||||
|
||||
bool isPublicationExist(pqxx::work & tx);
|
||||
|
||||
void createPublicationIfNeeded(pqxx::work & tx, bool create_without_check = false);
|
||||
|
||||
NameSet fetchTablesFromPublication(pqxx::work & tx);
|
||||
|
||||
void dropPublication(pqxx::nontransaction & ntx);
|
||||
|
||||
/// Methods to manage Replication Slots.
|
||||
|
||||
bool isReplicationSlotExist(pqxx::nontransaction & tx, String & start_lsn, bool temporary = false);
|
||||
|
||||
void createReplicationSlot(pqxx::nontransaction & tx, String & start_lsn, String & snapshot_name, bool temporary = false);
|
||||
|
||||
void dropReplicationSlot(pqxx::nontransaction & tx, bool temporary = false);
|
||||
|
||||
/// Methods to manage replication.
|
||||
|
||||
void waitConnectionAndStart();
|
||||
|
||||
void consumerFunc();
|
||||
|
||||
StoragePtr loadFromSnapshot(std::string & snapshot_name, const String & table_name, StorageMaterializedPostgreSQL * materialized_storage);
|
||||
|
||||
void reloadFromSnapshot(const std::vector<std::pair<Int32, String>> & relation_data);
|
||||
|
||||
PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name) const;
|
||||
|
||||
Poco::Logger * log;
|
||||
ContextPtr context;
|
||||
|
||||
const String remote_database_name, current_database_name;
|
||||
|
||||
/// Connection string and address for logs.
|
||||
postgres::ConnectionInfo connection_info;
|
||||
|
||||
/// max_block_size for replication stream.
|
||||
const size_t max_block_size;
|
||||
|
||||
/// Table structure changes are always tracked. By default, table with changed schema will get into a skip list.
|
||||
/// This setting allows to reloas table in the background.
|
||||
bool allow_automatic_update = false;
|
||||
|
||||
/// To distinguish whether current replication handler belongs to a MaterializedPostgreSQL database engine or single storage.
|
||||
bool is_materialized_postgresql_database;
|
||||
|
||||
/// A coma-separated list of tables, which are going to be replicated for database engine. By default, a whole database is replicated.
|
||||
String tables_list;
|
||||
|
||||
String replication_slot, publication_name;
|
||||
|
||||
/// Shared between replication_consumer and replication_handler, but never accessed concurrently.
|
||||
std::shared_ptr<postgres::Connection> connection;
|
||||
|
||||
/// Replication consumer. Manages decoding of replication stream and syncing into tables.
|
||||
std::shared_ptr<MaterializedPostgreSQLConsumer> consumer;
|
||||
|
||||
BackgroundSchedulePool::TaskHolder startup_task, consumer_task;
|
||||
|
||||
std::atomic<bool> stop_synchronization = false;
|
||||
|
||||
/// For database engine there are 2 places where it is checked for publication:
|
||||
/// 1. to fetch tables list from already created publication when database is loaded
|
||||
/// 2. at replication startup
|
||||
bool new_publication_created = false;
|
||||
|
||||
/// MaterializedPostgreSQL tables. Used for managing all operations with its internal nested tables.
|
||||
MaterializedStorages materialized_storages;
|
||||
|
||||
UInt64 milliseconds_to_wait;
|
||||
};
|
||||
|
||||
}
|
501
src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp
Normal file
501
src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp
Normal file
@ -0,0 +1,501 @@
|
||||
#include "StorageMaterializedPostgreSQL.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Common/Macros.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataStreams/ConvertingBlockInputStream.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Processors/Transforms/FilterTransform.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Processors/Sources/SourceFromInputStream.h>
|
||||
#include <Processors/Pipe.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Storages/ReadFinalForExternalReplicaStorage.h>
|
||||
#include <Core/PostgreSQL/Connection.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static const auto NESTED_TABLE_SUFFIX = "_nested";
|
||||
static const auto TMP_SUFFIX = "_tmp";
|
||||
|
||||
|
||||
/// For the case of single storage.
|
||||
StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(
|
||||
const StorageID & table_id_,
|
||||
bool is_attach_,
|
||||
const String & remote_database_name,
|
||||
const String & remote_table_name_,
|
||||
const postgres::ConnectionInfo & connection_info,
|
||||
const StorageInMemoryMetadata & storage_metadata,
|
||||
ContextPtr context_,
|
||||
std::unique_ptr<MaterializedPostgreSQLSettings> replication_settings)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, is_materialized_postgresql_database(false)
|
||||
, has_nested(false)
|
||||
, 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)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializedPostgreSQL is allowed only for Atomic database");
|
||||
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
String replication_identifier = remote_database_name + "_" + remote_table_name_;
|
||||
replication_handler = std::make_unique<PostgreSQLReplicationHandler>(
|
||||
replication_identifier,
|
||||
remote_database_name,
|
||||
table_id_.database_name,
|
||||
connection_info,
|
||||
getContext(),
|
||||
replication_settings->materialized_postgresql_max_block_size.value,
|
||||
/* allow_automatic_update */ false, /* is_materialized_postgresql_database */false);
|
||||
}
|
||||
|
||||
|
||||
/// 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.
|
||||
StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(const StorageID & table_id_, ContextPtr context_)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, is_materialized_postgresql_database(true)
|
||||
, has_nested(false)
|
||||
, nested_context(makeNestedTableContext(context_->getGlobalContext()))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
/// Constructor for MaterializedPostgreSQL table engine - for the case of MaterializePosgreSQL database engine.
|
||||
/// It is used when nested ReplacingMergeeTree table has already been created by replication thread.
|
||||
/// This storage is ready to handle read queries.
|
||||
StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL(StoragePtr nested_storage_, ContextPtr context_)
|
||||
: IStorage(nested_storage_->getStorageID())
|
||||
, WithContext(context_->getGlobalContext())
|
||||
, is_materialized_postgresql_database(true)
|
||||
, has_nested(true)
|
||||
, nested_context(makeNestedTableContext(context_->getGlobalContext()))
|
||||
, nested_table_id(nested_storage_->getStorageID())
|
||||
{
|
||||
setInMemoryMetadata(nested_storage_->getInMemoryMetadata());
|
||||
}
|
||||
|
||||
|
||||
/// 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.
|
||||
StoragePtr StorageMaterializedPostgreSQL::createTemporary() const
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
auto tmp_table_id = StorageID(table_id.database_name, table_id.table_name + TMP_SUFFIX);
|
||||
|
||||
/// 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());
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), getContext(), tmp_table_id, /* no delay */true);
|
||||
}
|
||||
|
||||
auto new_context = Context::createCopy(context);
|
||||
return StorageMaterializedPostgreSQL::create(tmp_table_id, new_context);
|
||||
}
|
||||
|
||||
|
||||
StoragePtr StorageMaterializedPostgreSQL::getNested() const
|
||||
{
|
||||
return DatabaseCatalog::instance().getTable(getNestedStorageID(), nested_context);
|
||||
}
|
||||
|
||||
|
||||
StoragePtr StorageMaterializedPostgreSQL::tryGetNested() const
|
||||
{
|
||||
return DatabaseCatalog::instance().tryGetTable(getNestedStorageID(), nested_context);
|
||||
}
|
||||
|
||||
|
||||
String StorageMaterializedPostgreSQL::getNestedTableName() const
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
|
||||
if (is_materialized_postgresql_database)
|
||||
return table_id.table_name;
|
||||
|
||||
return toString(table_id.uuid) + NESTED_TABLE_SUFFIX;
|
||||
}
|
||||
|
||||
|
||||
StorageID StorageMaterializedPostgreSQL::getNestedStorageID() const
|
||||
{
|
||||
if (nested_table_id.has_value())
|
||||
return nested_table_id.value();
|
||||
|
||||
auto table_id = getStorageID();
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"No storageID found for inner table. ({})", table_id.getNameForLogs());
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure)
|
||||
{
|
||||
const auto ast_create = getCreateNestedTableQuery(std::move(table_structure));
|
||||
auto table_id = getStorageID();
|
||||
auto tmp_nested_table_id = StorageID(table_id.database_name, getNestedTableName());
|
||||
|
||||
try
|
||||
{
|
||||
InterpreterCreateQuery interpreter(ast_create, nested_context);
|
||||
interpreter.execute();
|
||||
|
||||
auto nested_storage = DatabaseCatalog::instance().getTable(tmp_nested_table_id, nested_context);
|
||||
/// Save storage_id with correct uuid.
|
||||
nested_table_id = nested_storage->getStorageID();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while creating nested table: {}", tmp_nested_table_id.getNameForLogs());
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<Context> StorageMaterializedPostgreSQL::makeNestedTableContext(ContextPtr from_context)
|
||||
{
|
||||
auto new_context = Context::createCopy(from_context);
|
||||
new_context->setInternalQuery(true);
|
||||
return new_context;
|
||||
}
|
||||
|
||||
|
||||
StoragePtr StorageMaterializedPostgreSQL::prepare()
|
||||
{
|
||||
auto nested_table = getNested();
|
||||
setInMemoryMetadata(nested_table->getInMemoryMetadata());
|
||||
has_nested.store(true);
|
||||
return nested_table;
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedPostgreSQL::startup()
|
||||
{
|
||||
/// replication_handler != nullptr only in case of single table engine MaterializedPostgreSQL.
|
||||
if (replication_handler)
|
||||
{
|
||||
replication_handler->addStorage(remote_table_name, this);
|
||||
|
||||
if (is_attach)
|
||||
{
|
||||
/// In case of attach table use background startup in a separate thread. First wait until connection is reachable,
|
||||
/// then check for nested table -- it should already be created.
|
||||
replication_handler->startup();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Start synchronization preliminary setup immediately and throw in case of failure.
|
||||
/// It should be guaranteed that if MaterializedPostgreSQL table was created successfully, then
|
||||
/// its nested table was also created.
|
||||
replication_handler->startSynchronization(/* throw_on_error */ true);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedPostgreSQL::shutdown()
|
||||
{
|
||||
if (replication_handler)
|
||||
replication_handler->shutdown();
|
||||
auto nested = getNested();
|
||||
if (nested)
|
||||
nested->shutdown();
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedPostgreSQL::dropInnerTableIfAny(bool no_delay, ContextPtr local_context)
|
||||
{
|
||||
/// If it is a table with database engine MaterializedPostgreSQL - return, because delition of
|
||||
/// internal tables is managed there.
|
||||
if (is_materialized_postgresql_database)
|
||||
return;
|
||||
|
||||
replication_handler->shutdownFinal();
|
||||
|
||||
auto nested_table = getNested();
|
||||
if (nested_table)
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, getNestedStorageID(), no_delay);
|
||||
}
|
||||
|
||||
|
||||
NamesAndTypesList StorageMaterializedPostgreSQL::getVirtuals() const
|
||||
{
|
||||
return NamesAndTypesList{
|
||||
{"_sign", std::make_shared<DataTypeInt8>()},
|
||||
{"_version", std::make_shared<DataTypeUInt64>()}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
Pipe StorageMaterializedPostgreSQL::read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context_,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams)
|
||||
{
|
||||
auto materialized_table_lock = lockForShare(String(), context_->getSettingsRef().lock_acquire_timeout);
|
||||
auto nested_table = getNested();
|
||||
return readFinalFromNestedStorage(nested_table, column_names, metadata_snapshot,
|
||||
query_info, context_, processed_stage, max_block_size, num_streams);
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<ASTColumnDeclaration> StorageMaterializedPostgreSQL::getMaterializedColumnsDeclaration(
|
||||
const String name, const String type, UInt64 default_value)
|
||||
{
|
||||
auto column_declaration = std::make_shared<ASTColumnDeclaration>();
|
||||
|
||||
column_declaration->name = 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;
|
||||
}
|
||||
|
||||
|
||||
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
|
||||
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");
|
||||
}
|
||||
|
||||
return std::make_shared<ASTIdentifier>(data_type->getName());
|
||||
}
|
||||
|
||||
|
||||
/// 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
|
||||
/// transaction with snapshot, which is used for initial tables dump.
|
||||
ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure)
|
||||
{
|
||||
auto create_table_query = std::make_shared<ASTCreateQuery>();
|
||||
|
||||
auto table_id = getStorageID();
|
||||
create_table_query->table = getNestedTableName();
|
||||
create_table_query->database = table_id.database_name;
|
||||
if (is_materialized_postgresql_database)
|
||||
create_table_query->uuid = table_id.uuid;
|
||||
|
||||
auto columns_declare_list = std::make_shared<ASTColumns>();
|
||||
auto columns_expression_list = std::make_shared<ASTExpressionList>();
|
||||
auto order_by_expression = std::make_shared<ASTFunction>();
|
||||
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
const auto & columns = metadata_snapshot->getColumns();
|
||||
NamesAndTypesList ordinary_columns_and_types;
|
||||
|
||||
if (!is_materialized_postgresql_database)
|
||||
{
|
||||
ordinary_columns_and_types = columns.getOrdinary();
|
||||
}
|
||||
else
|
||||
{
|
||||
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)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"No columns returned for table {}.{}", table_id.database_name, table_id.table_name);
|
||||
}
|
||||
|
||||
ordinary_columns_and_types = *table_structure->columns;
|
||||
|
||||
if (!table_structure->primary_key_columns && !table_structure->replica_identity_columns)
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Table {}.{} has no primary key and no replica identity index", table_id.database_name, table_id.table_name);
|
||||
}
|
||||
|
||||
NamesAndTypesList merging_columns;
|
||||
if (table_structure->primary_key_columns)
|
||||
merging_columns = *table_structure->primary_key_columns;
|
||||
else
|
||||
merging_columns = *table_structure->replica_identity_columns;
|
||||
|
||||
order_by_expression->name = "tuple";
|
||||
order_by_expression->arguments = std::make_shared<ASTExpressionList>();
|
||||
|
||||
for (const auto & column : merging_columns)
|
||||
order_by_expression->arguments->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
|
||||
}
|
||||
|
||||
for (const auto & [name, type] : ordinary_columns_and_types)
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
columns_declare_list->set(columns_declare_list->columns, columns_expression_list);
|
||||
|
||||
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);
|
||||
|
||||
/// Not nullptr for single storage (because throws exception if not specified), nullptr otherwise.
|
||||
auto primary_key_ast = getInMemoryMetadataPtr()->getPrimaryKeyAST();
|
||||
|
||||
auto storage = std::make_shared<ASTStorage>();
|
||||
storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared<ASTIdentifier>("_version")));
|
||||
|
||||
if (primary_key_ast)
|
||||
storage->set(storage->order_by, primary_key_ast);
|
||||
else
|
||||
storage->set(storage->order_by, order_by_expression);
|
||||
|
||||
create_table_query->set(create_table_query->storage, storage);
|
||||
|
||||
/// 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));
|
||||
storage_metadata.setConstraints(metadata_snapshot->getConstraints());
|
||||
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
return create_table_query;
|
||||
}
|
||||
|
||||
|
||||
void registerStorageMaterializedPostgreSQL(StorageFactory & factory)
|
||||
{
|
||||
auto creator_fn = [](const StorageFactory::Arguments & args)
|
||||
{
|
||||
ASTs & engine_args = args.engine_args;
|
||||
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);
|
||||
|
||||
if (engine_args.size() != 5)
|
||||
throw Exception("Storage MaterializedPostgreSQL requires 5 parameters: "
|
||||
"PostgreSQL('host:port', 'database', 'table', 'username', 'password'",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (auto & engine_arg : engine_args)
|
||||
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getContext());
|
||||
|
||||
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("Storage MaterializedPostgreSQL needs order by key or primary key", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (args.storage_def->primary_key)
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.getContext());
|
||||
else
|
||||
metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->order_by->ptr(), metadata.columns, args.getContext());
|
||||
|
||||
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>();
|
||||
const String & remote_database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
/// No connection is made here, see Storages/PostgreSQL/PostgreSQLConnection.cpp
|
||||
auto connection_info = postgres::formatConnectionString(
|
||||
remote_database,
|
||||
parsed_host_port.first,
|
||||
parsed_host_port.second,
|
||||
engine_args[3]->as<ASTLiteral &>().value.safeGet<String>(),
|
||||
engine_args[4]->as<ASTLiteral &>().value.safeGet<String>());
|
||||
|
||||
return StorageMaterializedPostgreSQL::create(
|
||||
args.table_id, args.attach, remote_database, remote_table, connection_info,
|
||||
metadata, args.getContext(),
|
||||
std::move(postgresql_replication_settings));
|
||||
};
|
||||
|
||||
factory.registerStorage(
|
||||
"MaterializedPostgreSQL",
|
||||
creator_fn,
|
||||
StorageFactory::StorageFeatures{
|
||||
.supports_settings = true,
|
||||
.supports_sort_order = true,
|
||||
.source_access_type = AccessType::POSTGRES,
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
180
src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h
Normal file
180
src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h
Normal file
@ -0,0 +1,180 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include "PostgreSQLReplicationHandler.h"
|
||||
#include "MaterializedPostgreSQLSettings.h"
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <common/shared_ptr_helper.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Case of single MaterializedPostgreSQL table engine.
|
||||
*
|
||||
* A user creates a table with engine MaterializedPostgreSQL. Order by expression must be specified (needed for
|
||||
* nested ReplacingMergeTree table). This storage owns its own replication handler, which loads table data
|
||||
* from PostgreSQL into nested ReplacingMergeTree table. If table is not created, but attached, replication handler
|
||||
* will not start loading-from-snapshot procedure, instead it will continue from last committed lsn.
|
||||
*
|
||||
* Main point: Both tables exist on disk; database engine interacts only with the main table and main table takes
|
||||
* total ownershot over nested table. Nested table has name `main_table_uuid` + NESTED_SUFFIX.
|
||||
*
|
||||
**/
|
||||
|
||||
|
||||
/** Case of MaterializedPostgreSQL database engine.
|
||||
*
|
||||
* MaterializedPostgreSQL table exists only in memory and acts as a wrapper for nested table, i.e. only provides an
|
||||
* interface to work with nested table. Both tables share the same StorageID.
|
||||
*
|
||||
* Main table is never created or dropped via database method. The only way database engine interacts with
|
||||
* MaterializedPostgreSQL table - in tryGetTable() method, a MaterializedPostgreSQL table is returned in order to wrap
|
||||
* and redirect read requests. Set of such wrapper-tables is cached inside database engine. All other methods in
|
||||
* regard to materializePostgreSQL table are handled by replication handler.
|
||||
*
|
||||
* All database methods, apart from tryGetTable(), are devoted only to nested table.
|
||||
* NOTE: It makes sense to allow rename method for MaterializedPostgreSQL table via database method.
|
||||
* TODO: Make sure replication-to-table data channel is done only by relation_id.
|
||||
*
|
||||
* Also main table has the same InMemoryMetadata as its nested table, so if metadata of nested table changes - main table also has
|
||||
* to update its metadata, because all read requests are passed to MaterializedPostgreSQL table and then it redirects read
|
||||
* into nested table.
|
||||
*
|
||||
* When there is a need to update table structure, there will be created a new MaterializedPostgreSQL table with its own nested table,
|
||||
* it will have updated table schema and all data will be loaded from scratch in the background, while previous table with outadted table
|
||||
* structure will still serve read requests. When data is loaded, nested tables will be swapped, metadata of metarialzied table will be
|
||||
* updated according to nested table.
|
||||
*
|
||||
**/
|
||||
|
||||
class StorageMaterializedPostgreSQL final : public shared_ptr_helper<StorageMaterializedPostgreSQL>, public IStorage, WithContext
|
||||
{
|
||||
friend struct shared_ptr_helper<StorageMaterializedPostgreSQL>;
|
||||
|
||||
public:
|
||||
StorageMaterializedPostgreSQL(const StorageID & table_id_, ContextPtr context_);
|
||||
|
||||
StorageMaterializedPostgreSQL(StoragePtr nested_storage_, ContextPtr context_);
|
||||
|
||||
String getName() const override { return "MaterializedPostgreSQL"; }
|
||||
|
||||
void startup() override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
/// Used only for single MaterializedPostgreSQL storage.
|
||||
void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override;
|
||||
|
||||
NamesAndTypesList getVirtuals() const override;
|
||||
|
||||
Pipe read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context_,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
/// This method is called only from MateriaizePostgreSQL database engine, because it needs to maintain
|
||||
/// an invariant: a table exists only if its nested table exists. This atomic variable is set to _true_
|
||||
/// only once - when nested table is successfully created and is never changed afterwards.
|
||||
bool hasNested() { return has_nested.load(); }
|
||||
|
||||
void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure);
|
||||
|
||||
StoragePtr getNested() const;
|
||||
|
||||
StoragePtr tryGetNested() const;
|
||||
|
||||
/// Create a temporary MaterializedPostgreSQL table with current_table_name + TMP_SUFFIX.
|
||||
/// An empty wrapper is returned - it does not have inMemory metadata, just acts as an empty wrapper over
|
||||
/// temporary nested, which will be created shortly after.
|
||||
StoragePtr createTemporary() const;
|
||||
|
||||
ContextPtr getNestedTableContext() const { return nested_context; }
|
||||
|
||||
StorageID getNestedStorageID() const;
|
||||
|
||||
void setNestedStorageID(const StorageID & id) { nested_table_id.emplace(id); }
|
||||
|
||||
static std::shared_ptr<Context> makeNestedTableContext(ContextPtr from_context);
|
||||
|
||||
/// Get nested table (or throw if it does not exist), set in-memory metadata (taken from nested table)
|
||||
/// for current table, set has_nested = true.
|
||||
StoragePtr prepare();
|
||||
|
||||
protected:
|
||||
StorageMaterializedPostgreSQL(
|
||||
const StorageID & table_id_,
|
||||
bool is_attach_,
|
||||
const String & remote_database_name,
|
||||
const String & remote_table_name,
|
||||
const postgres::ConnectionInfo & connection_info,
|
||||
const StorageInMemoryMetadata & storage_metadata,
|
||||
ContextPtr context_,
|
||||
std::unique_ptr<MaterializedPostgreSQLSettings> replication_settings);
|
||||
|
||||
private:
|
||||
static std::shared_ptr<ASTColumnDeclaration> getMaterializedColumnsDeclaration(
|
||||
const String name, const String type, UInt64 default_value);
|
||||
|
||||
ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const;
|
||||
|
||||
ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure);
|
||||
|
||||
String getNestedTableName() const;
|
||||
|
||||
/// Not nullptr only for single MaterializedPostgreSQL storage, because for MaterializedPostgreSQL
|
||||
/// database engine there is one replication handler for all tables.
|
||||
std::unique_ptr<PostgreSQLReplicationHandler> replication_handler;
|
||||
|
||||
/// Distinguish between single MaterilizePostgreSQL table engine and MaterializedPostgreSQL database engine,
|
||||
/// because table with engine MaterilizePostgreSQL acts differently in each case.
|
||||
bool is_materialized_postgresql_database = false;
|
||||
|
||||
/// Will be set to `true` only once - when nested table was loaded by replication thread.
|
||||
/// After that, it will never be changed. Needed for MaterializedPostgreSQL database engine
|
||||
/// because there is an invariant - table exists only if its nested table exists, but nested
|
||||
/// table is not loaded immediately. It is made atomic, because it is accessed only by database engine,
|
||||
/// and updated by replication handler (only once).
|
||||
std::atomic<bool> has_nested = false;
|
||||
|
||||
/// Nested table context is a copy of global context, but modified to answer isInternalQuery() == true.
|
||||
/// This is needed to let database engine know whether to access nested table or a wrapper over nested (materialized table).
|
||||
ContextMutablePtr nested_context;
|
||||
|
||||
/// Save nested storageID to be able to fetch it. It is set once nested is created and will be
|
||||
/// updated only when nested is reloaded or renamed.
|
||||
std::optional<StorageID> nested_table_id;
|
||||
|
||||
/// Needed only for the case of single MaterializedPostgreSQL storage - in order to make
|
||||
/// delayed storage forwarding into replication handler.
|
||||
String remote_table_name;
|
||||
|
||||
/// Needed only for the case of single MaterializedPostgreSQL storage, because in case of create
|
||||
/// query (not attach) initial setup will be done immediately and error message is thrown at once.
|
||||
/// It results in the fact: single MaterializedPostgreSQL storage is created only if its nested table is created.
|
||||
/// In case of attach - this setup will be done in a separate thread in the background. It will also
|
||||
/// be checked for nested table and attempted to load it if it does not exist for some reason.
|
||||
bool is_attach = true;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
86
src/Storages/ReadFinalForExternalReplicaStorage.cpp
Normal file
86
src/Storages/ReadFinalForExternalReplicaStorage.cpp
Normal file
@ -0,0 +1,86 @@
|
||||
#include <Storages/ReadFinalForExternalReplicaStorage.h>
|
||||
|
||||
#if USE_MYSQL || USE_LIBPQXX
|
||||
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Processors/Transforms/FilterTransform.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Pipe readFinalFromNestedStorage(
|
||||
StoragePtr nested_storage,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned int num_streams)
|
||||
{
|
||||
NameSet column_names_set = NameSet(column_names.begin(), column_names.end());
|
||||
auto lock = nested_storage->lockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
||||
const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr();
|
||||
|
||||
Block nested_header = nested_metadata->getSampleBlock();
|
||||
ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2);
|
||||
ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1);
|
||||
|
||||
if (ASTSelectQuery * select_query = query_info.query->as<ASTSelectQuery>(); select_query && !column_names_set.count(version_column.name))
|
||||
{
|
||||
auto & tables_in_select_query = select_query->tables()->as<ASTTablesInSelectQuery &>();
|
||||
|
||||
if (!tables_in_select_query.children.empty())
|
||||
{
|
||||
auto & tables_element = tables_in_select_query.children[0]->as<ASTTablesInSelectQueryElement &>();
|
||||
|
||||
if (tables_element.table_expression)
|
||||
tables_element.table_expression->as<ASTTableExpression &>().final = true;
|
||||
}
|
||||
}
|
||||
|
||||
String filter_column_name;
|
||||
Names require_columns_name = column_names;
|
||||
ASTPtr expressions = std::make_shared<ASTExpressionList>();
|
||||
if (column_names_set.empty() || !column_names_set.count(sign_column.name))
|
||||
{
|
||||
require_columns_name.emplace_back(sign_column.name);
|
||||
|
||||
const auto & sign_column_name = std::make_shared<ASTIdentifier>(sign_column.name);
|
||||
const auto & fetch_sign_value = std::make_shared<ASTLiteral>(Field(Int8(1)));
|
||||
|
||||
expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value));
|
||||
filter_column_name = expressions->children.back()->getColumnName();
|
||||
|
||||
for (const auto & column_name : column_names)
|
||||
expressions->children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
|
||||
}
|
||||
|
||||
Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
pipe.addTableLock(lock);
|
||||
|
||||
if (!expressions->children.empty() && !pipe.empty())
|
||||
{
|
||||
Block pipe_header = pipe.getHeader();
|
||||
auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList());
|
||||
ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true /* add_aliases */, false /* project_result */);
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<FilterTransform>(header, expression_actions, filter_column_name, false);
|
||||
});
|
||||
}
|
||||
|
||||
return pipe;
|
||||
}
|
||||
}
|
||||
|
||||
#endif
|
28
src/Storages/ReadFinalForExternalReplicaStorage.h
Normal file
28
src/Storages/ReadFinalForExternalReplicaStorage.h
Normal file
@ -0,0 +1,28 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
# include "config_core.h"
|
||||
#endif
|
||||
|
||||
#if USE_MYSQL || USE_LIBPQXX
|
||||
|
||||
#include <Storages/StorageProxy.h>
|
||||
#include <Processors/Pipe.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Pipe readFinalFromNestedStorage(
|
||||
StoragePtr nested_storage,
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
size_t max_block_size,
|
||||
unsigned int num_streams);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -22,6 +22,7 @@
|
||||
#include <Processors/Transforms/FilterTransform.h>
|
||||
|
||||
#include <Databases/MySQL/DatabaseMaterializeMySQL.h>
|
||||
#include <Storages/ReadFinalForExternalReplicaStorage.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
|
||||
namespace DB
|
||||
@ -37,7 +38,7 @@ StorageMaterializeMySQL::StorageMaterializeMySQL(const StoragePtr & nested_stora
|
||||
|
||||
Pipe StorageMaterializeMySQL::read(
|
||||
const Names & column_names,
|
||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
SelectQueryInfo & query_info,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
@ -46,61 +47,8 @@ Pipe StorageMaterializeMySQL::read(
|
||||
{
|
||||
/// If the background synchronization thread has exception.
|
||||
rethrowSyncExceptionIfNeed(database);
|
||||
|
||||
NameSet column_names_set = NameSet(column_names.begin(), column_names.end());
|
||||
auto lock = nested_storage->lockForShare(context->getCurrentQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
||||
const StorageMetadataPtr & nested_metadata = nested_storage->getInMemoryMetadataPtr();
|
||||
|
||||
Block nested_header = nested_metadata->getSampleBlock();
|
||||
ColumnWithTypeAndName & sign_column = nested_header.getByPosition(nested_header.columns() - 2);
|
||||
ColumnWithTypeAndName & version_column = nested_header.getByPosition(nested_header.columns() - 1);
|
||||
|
||||
if (ASTSelectQuery * select_query = query_info.query->as<ASTSelectQuery>(); select_query && !column_names_set.count(version_column.name))
|
||||
{
|
||||
auto & tables_in_select_query = select_query->tables()->as<ASTTablesInSelectQuery &>();
|
||||
|
||||
if (!tables_in_select_query.children.empty())
|
||||
{
|
||||
auto & tables_element = tables_in_select_query.children[0]->as<ASTTablesInSelectQueryElement &>();
|
||||
|
||||
if (tables_element.table_expression)
|
||||
tables_element.table_expression->as<ASTTableExpression &>().final = true;
|
||||
}
|
||||
}
|
||||
|
||||
String filter_column_name;
|
||||
Names require_columns_name = column_names;
|
||||
ASTPtr expressions = std::make_shared<ASTExpressionList>();
|
||||
if (column_names_set.empty() || !column_names_set.count(sign_column.name))
|
||||
{
|
||||
require_columns_name.emplace_back(sign_column.name);
|
||||
|
||||
const auto & sign_column_name = std::make_shared<ASTIdentifier>(sign_column.name);
|
||||
const auto & fetch_sign_value = std::make_shared<ASTLiteral>(Field(Int8(1)));
|
||||
|
||||
expressions->children.emplace_back(makeASTFunction("equals", sign_column_name, fetch_sign_value));
|
||||
filter_column_name = expressions->children.back()->getColumnName();
|
||||
|
||||
for (const auto & column_name : column_names)
|
||||
expressions->children.emplace_back(std::make_shared<ASTIdentifier>(column_name));
|
||||
}
|
||||
|
||||
Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams);
|
||||
pipe.addTableLock(lock);
|
||||
|
||||
if (!expressions->children.empty() && !pipe.empty())
|
||||
{
|
||||
Block pipe_header = pipe.getHeader();
|
||||
auto syntax = TreeRewriter(context).analyze(expressions, pipe_header.getNamesAndTypesList());
|
||||
ExpressionActionsPtr expression_actions = ExpressionAnalyzer(expressions, syntax, context).getActions(true /* add_aliases */, false /* project_result */);
|
||||
|
||||
pipe.addSimpleTransform([&](const Block & header)
|
||||
{
|
||||
return std::make_shared<FilterTransform>(header, expression_actions, filter_column_name, false);
|
||||
});
|
||||
}
|
||||
|
||||
return pipe;
|
||||
return readFinalFromNestedStorage(nested_storage, column_names, metadata_snapshot,
|
||||
query_info, context, processed_stage, max_block_size, num_streams);
|
||||
}
|
||||
|
||||
NamesAndTypesList StorageMaterializeMySQL::getVirtuals() const
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
@ -229,36 +228,6 @@ BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const
|
||||
}
|
||||
|
||||
|
||||
static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, const StorageID & target_table_id, bool no_delay)
|
||||
{
|
||||
if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context))
|
||||
{
|
||||
/// We create and execute `drop` query for internal table.
|
||||
auto drop_query = std::make_shared<ASTDropQuery>();
|
||||
drop_query->database = target_table_id.database_name;
|
||||
drop_query->table = target_table_id.table_name;
|
||||
drop_query->kind = kind;
|
||||
drop_query->no_delay = no_delay;
|
||||
drop_query->if_exists = true;
|
||||
ASTPtr ast_drop_query = drop_query;
|
||||
/// FIXME We have to use global context to execute DROP query for inner table
|
||||
/// to avoid "Not enough privileges" error if current user has only DROP VIEW ON mat_view_name privilege
|
||||
/// and not allowed to drop inner table explicitly. Allowing to drop inner table without explicit grant
|
||||
/// looks like expected behaviour and we have tests for it.
|
||||
auto drop_context = Context::createCopy(global_context);
|
||||
drop_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
|
||||
if (auto txn = current_context->getZooKeeperMetadataTransaction())
|
||||
{
|
||||
/// For Replicated database
|
||||
drop_context->setQueryContext(std::const_pointer_cast<Context>(current_context));
|
||||
drop_context->initZooKeeperMetadataTransaction(txn, true);
|
||||
}
|
||||
InterpreterDropQuery drop_interpreter(ast_drop_query, drop_context);
|
||||
drop_interpreter.execute();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedView::drop()
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
@ -266,19 +235,19 @@ void StorageMaterializedView::drop()
|
||||
if (!select_query.select_table_id.empty())
|
||||
DatabaseCatalog::instance().removeDependency(select_query.select_table_id, table_id);
|
||||
|
||||
dropInnerTable(true, getContext());
|
||||
dropInnerTableIfAny(true, getContext());
|
||||
}
|
||||
|
||||
void StorageMaterializedView::dropInnerTable(bool no_delay, ContextPtr local_context)
|
||||
void StorageMaterializedView::dropInnerTableIfAny(bool no_delay, ContextPtr local_context)
|
||||
{
|
||||
if (has_inner_table && tryGetTargetTable())
|
||||
executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, no_delay);
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, no_delay);
|
||||
}
|
||||
|
||||
void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &)
|
||||
{
|
||||
if (has_inner_table)
|
||||
executeDropQuery(ASTDropQuery::Kind::Truncate, getContext(), local_context, target_table_id, true);
|
||||
InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Truncate, getContext(), local_context, target_table_id, true);
|
||||
}
|
||||
|
||||
void StorageMaterializedView::checkStatementCanBeForwarded() const
|
||||
|
@ -37,7 +37,7 @@ public:
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
|
||||
|
||||
void drop() override;
|
||||
void dropInnerTable(bool no_delay, ContextPtr context);
|
||||
void dropInnerTableIfAny(bool no_delay, ContextPtr local_context) override;
|
||||
|
||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include "StoragePostgreSQL.h"
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <DataStreams/PostgreSQLBlockInputStream.h>
|
||||
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Storages/transformQueryForExternalDatabase.h>
|
||||
@ -16,7 +17,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <DataStreams/PostgreSQLBlockInputStream.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include <Common/assert_cast.h>
|
||||
@ -90,7 +90,7 @@ Pipe StoragePostgreSQL::read(
|
||||
}
|
||||
|
||||
return Pipe(std::make_shared<SourceFromInputStream>(
|
||||
std::make_shared<PostgreSQLBlockInputStream>(pool->get(), query, sample_block, max_block_size_)));
|
||||
std::make_shared<PostgreSQLBlockInputStream<>>(pool->get(), query, sample_block, max_block_size_)));
|
||||
}
|
||||
|
||||
|
||||
|
@ -9,7 +9,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <Storages/PostgreSQL/PoolWithFailover.h>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -60,6 +60,7 @@ void registerStorageEmbeddedRocksDB(StorageFactory & factory);
|
||||
|
||||
#if USE_LIBPQXX
|
||||
void registerStoragePostgreSQL(StorageFactory & factory);
|
||||
void registerStorageMaterializedPostgreSQL(StorageFactory & factory);
|
||||
#endif
|
||||
|
||||
#if USE_MYSQL || USE_LIBPQXX
|
||||
@ -121,6 +122,7 @@ void registerStorages()
|
||||
|
||||
#if USE_LIBPQXX
|
||||
registerStoragePostgreSQL(factory);
|
||||
registerStorageMaterializedPostgreSQL(factory);
|
||||
#endif
|
||||
|
||||
#if USE_MYSQL || USE_LIBPQXX
|
||||
|
@ -1,6 +1,9 @@
|
||||
#include <TableFunctions/TableFunctionPostgreSQL.h>
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
@ -9,10 +12,8 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/parseAddress.h>
|
||||
#include "registerTableFunctions.h"
|
||||
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/parseRemoteDescription.h>
|
||||
#include <Storages/StoragePostgreSQL.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -47,11 +48,12 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr context) const
|
||||
{
|
||||
const bool use_nulls = context->getSettingsRef().external_table_functions_use_nulls;
|
||||
auto connection_holder = connection_pool->get();
|
||||
auto columns = fetchPostgreSQLTableStructure(
|
||||
connection_pool->get(),
|
||||
connection_holder->get(),
|
||||
remote_table_schema.empty() ? doubleQuoteString(remote_table_name)
|
||||
: doubleQuoteString(remote_table_schema) + '.' + doubleQuoteString(remote_table_name),
|
||||
use_nulls);
|
||||
use_nulls).columns;
|
||||
|
||||
return ColumnsDescription{*columns};
|
||||
}
|
||||
|
@ -5,7 +5,7 @@
|
||||
|
||||
#if USE_LIBPQXX
|
||||
#include <TableFunctions/ITableFunction.h>
|
||||
#include <Storages/PostgreSQL/PoolWithFailover.h>
|
||||
#include <Core/PostgreSQL/PoolWithFailover.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -0,0 +1,11 @@
|
||||
<yandex>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
</yandex>
|
@ -0,0 +1,8 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
<allow_experimental_database_materialized_postgresql>1</allow_experimental_database_materialized_postgresql>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
@ -0,0 +1,886 @@
|
||||
import pytest
|
||||
import time
|
||||
import psycopg2
|
||||
import os.path as p
|
||||
import random
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
from random import randrange
|
||||
import threading
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance',
|
||||
main_configs = ['configs/log_conf.xml'],
|
||||
user_configs = ['configs/users.xml'],
|
||||
with_postgres=True, stay_alive=True)
|
||||
|
||||
postgres_table_template = """
|
||||
CREATE TABLE IF NOT EXISTS {} (
|
||||
key Integer NOT NULL, value Integer, PRIMARY KEY(key))
|
||||
"""
|
||||
postgres_table_template_2 = """
|
||||
CREATE TABLE IF NOT EXISTS {} (
|
||||
key Integer NOT NULL, value1 Integer, value2 Integer, value3 Integer, PRIMARY KEY(key))
|
||||
"""
|
||||
postgres_table_template_3 = """
|
||||
CREATE TABLE IF NOT EXISTS {} (
|
||||
key1 Integer NOT NULL, value1 Integer, key2 Integer NOT NULL, value2 Integer NOT NULL)
|
||||
"""
|
||||
|
||||
def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name='postgres_database'):
|
||||
if database == True:
|
||||
conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format(ip, port, database_name)
|
||||
else:
|
||||
conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(ip, port)
|
||||
|
||||
conn = psycopg2.connect(conn_string)
|
||||
if auto_commit:
|
||||
conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT)
|
||||
conn.autocommit = True
|
||||
return conn
|
||||
|
||||
|
||||
def create_postgres_db(cursor, name='postgres_database'):
|
||||
cursor.execute("CREATE DATABASE {}".format(name))
|
||||
|
||||
def drop_postgres_db(cursor, name='postgres_database'):
|
||||
cursor.execute("DROP DATABASE {}".format(name))
|
||||
|
||||
def create_clickhouse_postgres_db(ip, port, name='postgres_database'):
|
||||
instance.query('''
|
||||
CREATE DATABASE {}
|
||||
ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')'''.format(name, ip, port, name))
|
||||
|
||||
def drop_clickhouse_postgres_db(name='postgres_database'):
|
||||
instance.query('DROP DATABASE {}'.format(name))
|
||||
|
||||
def create_materialized_db(ip, port,
|
||||
materialized_database='test_database',
|
||||
postgres_database='postgres_database',
|
||||
settings=[]):
|
||||
create_query = "CREATE DATABASE {} ENGINE = MaterializedPostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')".format(materialized_database, ip, port, postgres_database)
|
||||
if len(settings) > 0:
|
||||
create_query += " SETTINGS "
|
||||
for i in range(len(settings)):
|
||||
if i != 0:
|
||||
create_query += ', '
|
||||
create_query += settings[i]
|
||||
instance.query(create_query)
|
||||
assert materialized_database in instance.query('SHOW DATABASES')
|
||||
|
||||
def drop_materialized_db(materialized_database='test_database'):
|
||||
instance.query('DROP DATABASE IF EXISTS {}'.format(materialized_database))
|
||||
assert materialized_database not in instance.query('SHOW DATABASES')
|
||||
|
||||
def create_postgres_table(cursor, table_name, replica_identity_full=False, template=postgres_table_template):
|
||||
cursor.execute("DROP TABLE IF EXISTS {}".format(table_name))
|
||||
cursor.execute(template.format(table_name))
|
||||
if replica_identity_full:
|
||||
cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name))
|
||||
|
||||
queries = [
|
||||
'INSERT INTO postgresql_replica_{} select i, i from generate_series(0, 10000) as t(i);',
|
||||
'DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;',
|
||||
"UPDATE postgresql_replica_{} SET key=key+20000 WHERE key%2=0",
|
||||
'INSERT INTO postgresql_replica_{} select i, i from generate_series(40000, 50000) as t(i);',
|
||||
'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value + 101 WHERE key % 2 = 1;',
|
||||
"UPDATE postgresql_replica_{} SET key=key+80000 WHERE key%2=1",
|
||||
'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;',
|
||||
'INSERT INTO postgresql_replica_{} select i, i from generate_series(200000, 250000) as t(i);',
|
||||
'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 = 0;',
|
||||
"UPDATE postgresql_replica_{} SET key=key+500000 WHERE key%2=1",
|
||||
'INSERT INTO postgresql_replica_{} select i, i from generate_series(1000000, 1050000) as t(i);',
|
||||
'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;',
|
||||
"UPDATE postgresql_replica_{} SET key=key+10000000",
|
||||
'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;',
|
||||
'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;'
|
||||
]
|
||||
|
||||
|
||||
def assert_nested_table_is_created(table_name, materialized_database='test_database'):
|
||||
database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database))
|
||||
while table_name not in database_tables:
|
||||
time.sleep(0.2)
|
||||
database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database))
|
||||
assert(table_name in database_tables)
|
||||
|
||||
|
||||
def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database'):
|
||||
assert_nested_table_is_created(table_name, materialized_database)
|
||||
|
||||
expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by))
|
||||
result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by))
|
||||
|
||||
while result != expected:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by))
|
||||
|
||||
assert(result == expected)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
conn = get_postgres_conn(ip=cluster.postgres_ip,
|
||||
port=cluster.postgres_port)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_db(cursor, 'postgres_database')
|
||||
create_clickhouse_postgres_db(ip=cluster.postgres_ip,
|
||||
port=cluster.postgres_port)
|
||||
|
||||
instance.query("DROP DATABASE IF EXISTS test_database")
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def test_load_and_sync_all_database_tables(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 5
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
create_postgres_table(cursor, table_name);
|
||||
instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(50)".format(table_name))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
assert 'test_database' in instance.query('SHOW DATABASES')
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
check_tables_are_synchronized(table_name);
|
||||
cursor.execute('drop table {};'.format(table_name))
|
||||
|
||||
result = instance.query('''SELECT count() FROM system.tables WHERE database = 'test_database';''')
|
||||
assert(int(result) == NUM_TABLES)
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_replicating_dml(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 5
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i));
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format(i, i))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(1000)".format(i, i))
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
check_tables_are_synchronized(table_name);
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
cursor.execute('UPDATE postgresql_replica_{} SET value = {} * {} WHERE key < 50;'.format(i, i, i))
|
||||
cursor.execute('UPDATE postgresql_replica_{} SET value = {} * {} * {} WHERE key >= 50;'.format(i, i, i, i))
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
cursor.execute('DELETE FROM postgresql_replica_{} WHERE (value*value + {}) % 2 = 0;'.format(i, i))
|
||||
cursor.execute('UPDATE postgresql_replica_{} SET value = value - (value % 7) WHERE key > 128 AND key < 512;'.format(i))
|
||||
cursor.execute('DELETE FROM postgresql_replica_{} WHERE key % 7 = 1;'.format(i, i))
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
cursor.execute('drop table postgresql_replica_{};'.format(i))
|
||||
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_different_data_types(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
cursor.execute('drop table if exists test_data_types;')
|
||||
cursor.execute('drop table if exists test_array_data_type;')
|
||||
|
||||
cursor.execute(
|
||||
'''CREATE TABLE test_data_types (
|
||||
id integer PRIMARY KEY, a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial,
|
||||
h timestamp, i date, j decimal(5, 5), k numeric(5, 5))''')
|
||||
|
||||
cursor.execute(
|
||||
'''CREATE TABLE test_array_data_type
|
||||
(
|
||||
key Integer NOT NULL PRIMARY KEY,
|
||||
a Date[] NOT NULL, -- Date
|
||||
b Timestamp[] NOT NULL, -- DateTime
|
||||
c real[][] NOT NULL, -- Float32
|
||||
d double precision[][] NOT NULL, -- Float64
|
||||
e decimal(5, 5)[][][] NOT NULL, -- Decimal32
|
||||
f integer[][][] NOT NULL, -- Int32
|
||||
g Text[][][][][] NOT NULL, -- String
|
||||
h Integer[][][], -- Nullable(Int32)
|
||||
i Char(2)[][][][], -- Nullable(String)
|
||||
k Char(2)[] -- Nullable(String)
|
||||
)''')
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for i in range(10):
|
||||
instance.query('''
|
||||
INSERT INTO postgres_database.test_data_types VALUES
|
||||
({}, -32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.2, 0.2)'''.format(i))
|
||||
|
||||
check_tables_are_synchronized('test_data_types', 'id');
|
||||
result = instance.query('SELECT * FROM test_database.test_data_types ORDER BY id LIMIT 1;')
|
||||
assert(result == '0\t-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.20000\t0.20000\n')
|
||||
|
||||
for i in range(10):
|
||||
col = random.choice(['a', 'b', 'c'])
|
||||
cursor.execute('UPDATE test_data_types SET {} = {};'.format(col, i))
|
||||
cursor.execute('''UPDATE test_data_types SET i = '2020-12-12';'''.format(col, i))
|
||||
|
||||
check_tables_are_synchronized('test_data_types', 'id');
|
||||
cursor.execute('drop table test_data_types;')
|
||||
|
||||
instance.query("INSERT INTO postgres_database.test_array_data_type "
|
||||
"VALUES ("
|
||||
"0, "
|
||||
"['2000-05-12', '2000-05-12'], "
|
||||
"['2000-05-12 12:12:12', '2000-05-12 12:12:12'], "
|
||||
"[[1.12345], [1.12345], [1.12345]], "
|
||||
"[[1.1234567891], [1.1234567891], [1.1234567891]], "
|
||||
"[[[0.11111, 0.11111]], [[0.22222, 0.22222]], [[0.33333, 0.33333]]], "
|
||||
"[[[1, 1], [1, 1]], [[3, 3], [3, 3]], [[4, 4], [5, 5]]], "
|
||||
"[[[[['winx', 'winx', 'winx']]]]], "
|
||||
"[[[1, NULL], [NULL, 1]], [[NULL, NULL], [NULL, NULL]], [[4, 4], [5, 5]]], "
|
||||
"[[[[NULL]]]], "
|
||||
"[]"
|
||||
")")
|
||||
|
||||
expected = (
|
||||
"0\t" +
|
||||
"['2000-05-12','2000-05-12']\t" +
|
||||
"['2000-05-12 12:12:12','2000-05-12 12:12:12']\t" +
|
||||
"[[1.12345],[1.12345],[1.12345]]\t" +
|
||||
"[[1.1234567891],[1.1234567891],[1.1234567891]]\t" +
|
||||
"[[[0.11111,0.11111]],[[0.22222,0.22222]],[[0.33333,0.33333]]]\t"
|
||||
"[[[1,1],[1,1]],[[3,3],[3,3]],[[4,4],[5,5]]]\t"
|
||||
"[[[[['winx','winx','winx']]]]]\t"
|
||||
"[[[1,NULL],[NULL,1]],[[NULL,NULL],[NULL,NULL]],[[4,4],[5,5]]]\t"
|
||||
"[[[[NULL]]]]\t"
|
||||
"[]\n"
|
||||
)
|
||||
|
||||
check_tables_are_synchronized('test_array_data_type');
|
||||
result = instance.query('SELECT * FROM test_database.test_array_data_type ORDER BY key;')
|
||||
assert(result == expected)
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_load_and_sync_subset_of_database_tables(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 10
|
||||
|
||||
publication_tables = ''
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i));
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(50)".format(i))
|
||||
|
||||
if i < int(NUM_TABLES/2):
|
||||
if publication_tables != '':
|
||||
publication_tables += ', '
|
||||
publication_tables += table_name
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
settings=["materialized_postgresql_tables_list = '{}'".format(publication_tables)])
|
||||
assert 'test_database' in instance.query('SHOW DATABASES')
|
||||
|
||||
time.sleep(1)
|
||||
|
||||
for i in range(int(NUM_TABLES/2)):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
assert_nested_table_is_created(table_name)
|
||||
|
||||
result = instance.query('''SELECT count() FROM system.tables WHERE database = 'test_database';''')
|
||||
assert(int(result) == int(NUM_TABLES/2))
|
||||
|
||||
database_tables = instance.query('SHOW TABLES FROM test_database')
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
if i < int(NUM_TABLES/2):
|
||||
assert table_name in database_tables
|
||||
else:
|
||||
assert table_name not in database_tables
|
||||
instance.query("INSERT INTO postgres_database.{} SELECT 50 + number, {} from numbers(100)".format(table_name, i))
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
if i < int(NUM_TABLES/2):
|
||||
check_tables_are_synchronized(table_name);
|
||||
cursor.execute('drop table {};'.format(table_name))
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_changing_replica_identity_value(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, number from numbers(50)")
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 100 + number, number from numbers(50)")
|
||||
check_tables_are_synchronized('postgresql_replica');
|
||||
cursor.execute("UPDATE postgresql_replica SET key=key-25 WHERE key<100 ")
|
||||
check_tables_are_synchronized('postgresql_replica');
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_clickhouse_restart(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 5
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i));
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {} from numbers(50)".format(i, i))
|
||||
|
||||
instance.query("CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres', 'mysecretpassword')")
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
check_tables_are_synchronized(table_name);
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 50 + number, {} from numbers(50000)".format(i, i))
|
||||
|
||||
instance.restart_clickhouse()
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_replica_identity_index(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
|
||||
create_postgres_table(cursor, 'postgresql_replica', template=postgres_table_template_3);
|
||||
cursor.execute("CREATE unique INDEX idx on postgresql_replica(key1, key2);")
|
||||
cursor.execute("ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx")
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(50, 10)")
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(100, 10)")
|
||||
check_tables_are_synchronized('postgresql_replica', order_by='key1');
|
||||
|
||||
cursor.execute("UPDATE postgresql_replica SET key1=key1-25 WHERE key1<100 ")
|
||||
cursor.execute("UPDATE postgresql_replica SET key2=key2-25 WHERE key2>100 ")
|
||||
cursor.execute("UPDATE postgresql_replica SET value1=value1+100 WHERE key1<100 ")
|
||||
cursor.execute("UPDATE postgresql_replica SET value2=value2+200 WHERE key2>100 ")
|
||||
check_tables_are_synchronized('postgresql_replica', order_by='key1');
|
||||
|
||||
cursor.execute('DELETE FROM postgresql_replica WHERE key2<75;')
|
||||
check_tables_are_synchronized('postgresql_replica', order_by='key1');
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_table_schema_changes(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 5
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i), template=postgres_table_template_2);
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT number, {}, {}, {} from numbers(25)".format(i, i, i, i))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
settings=["materialized_postgresql_allow_automatic_update = 1"])
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 25 + number, {}, {}, {} from numbers(25)".format(i, i, i, i))
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
|
||||
expected = instance.query("SELECT key, value1, value3 FROM test_database.postgresql_replica_3 ORDER BY key");
|
||||
|
||||
altered_table = random.randint(0, 4)
|
||||
cursor.execute("ALTER TABLE postgresql_replica_{} DROP COLUMN value2".format(altered_table))
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
cursor.execute("INSERT INTO postgresql_replica_{} VALUES (50, {}, {})".format(i, i, i))
|
||||
cursor.execute("UPDATE postgresql_replica_{} SET value3 = 12 WHERE key%2=0".format(i))
|
||||
|
||||
assert_nested_table_is_created('postgresql_replica_{}'.format(altered_table))
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table))
|
||||
print('check1 OK')
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
if i != altered_table:
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {}, {} from numbers(49)".format(i, i, i, i))
|
||||
else:
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_{} SELECT 51 + number, {}, {} from numbers(49)".format(i, i, i))
|
||||
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(altered_table));
|
||||
print('check2 OK')
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
cursor.execute('drop table postgresql_replica_{};'.format(i))
|
||||
|
||||
instance.query("DROP DATABASE test_database")
|
||||
|
||||
|
||||
def test_many_concurrent_queries(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 5
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i));
|
||||
instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT number, number from numbers(10000)'.format(i))
|
||||
n = [10000]
|
||||
|
||||
query_pool = ['DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value - 125 WHERE key % 2 = 0;',
|
||||
'DELETE FROM postgresql_replica_{} WHERE key % 10 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value*5 WHERE key % 2 = 1;',
|
||||
'DELETE FROM postgresql_replica_{} WHERE value % 2 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value + 2000 WHERE key % 5 = 0;',
|
||||
'DELETE FROM postgresql_replica_{} WHERE value % 3 = 0;',
|
||||
'UPDATE postgresql_replica_{} SET value = value * 2 WHERE key % 3 = 0;',
|
||||
'DELETE FROM postgresql_replica_{} WHERE value % 9 = 2;',
|
||||
'UPDATE postgresql_replica_{} SET value = value + 2 WHERE key % 3 = 1;',
|
||||
'DELETE FROM postgresql_replica_{} WHERE value%5 = 0;']
|
||||
|
||||
def attack(thread_id):
|
||||
print('thread {}'.format(thread_id))
|
||||
k = 10000
|
||||
for i in range(20):
|
||||
query_id = random.randrange(0, len(query_pool)-1)
|
||||
table_id = random.randrange(0, 5) # num tables
|
||||
|
||||
# random update / delete query
|
||||
cursor.execute(query_pool[query_id].format(table_id))
|
||||
print("table {} query {} ok".format(table_id, query_id))
|
||||
|
||||
# allow some thread to do inserts (not to violate key constraints)
|
||||
if thread_id < 5:
|
||||
print("try insert table {}".format(thread_id))
|
||||
instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT {}*10000*({} + number), number from numbers(1000)'.format(i, thread_id, k))
|
||||
k += 1
|
||||
print("insert table {} ok".format(thread_id))
|
||||
|
||||
if i == 5:
|
||||
# also change primary key value
|
||||
print("try update primary key {}".format(thread_id))
|
||||
cursor.execute("UPDATE postgresql_replica_{} SET key=key%100000+100000*{} WHERE key%{}=0".format(thread_id, i+1, i+1))
|
||||
print("update primary key {} ok".format(thread_id))
|
||||
|
||||
threads = []
|
||||
threads_num = 16
|
||||
for i in range(threads_num):
|
||||
threads.append(threading.Thread(target=attack, args=(i,)))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for thread in threads:
|
||||
time.sleep(random.uniform(0, 1))
|
||||
thread.start()
|
||||
|
||||
n[0] = 50000
|
||||
for table_id in range(NUM_TABLES):
|
||||
n[0] += 1
|
||||
instance.query('INSERT INTO postgres_database.postgresql_replica_{} SELECT {} + number, number from numbers(5000)'.format(table_id, n[0]))
|
||||
#cursor.execute("UPDATE postgresql_replica_{} SET key=key%100000+100000*{} WHERE key%{}=0".format(table_id, table_id+1, table_id+1))
|
||||
|
||||
for thread in threads:
|
||||
thread.join()
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i))
|
||||
count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i))
|
||||
assert(int(count1) == int(count2))
|
||||
print(count1, count2)
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_single_transaction(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, auto_commit=False)
|
||||
cursor = conn.cursor()
|
||||
|
||||
create_postgres_table(cursor, 'postgresql_replica_0');
|
||||
conn.commit()
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
assert_nested_table_is_created('postgresql_replica_0')
|
||||
|
||||
for query in queries:
|
||||
print('query {}'.format(query))
|
||||
cursor.execute(query.format(0))
|
||||
|
||||
time.sleep(5)
|
||||
result = instance.query("select count() from test_database.postgresql_replica_0")
|
||||
# no commit yet
|
||||
assert(int(result) == 0)
|
||||
|
||||
conn.commit()
|
||||
check_tables_are_synchronized('postgresql_replica_0');
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_virtual_columns(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica_0');
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
settings=["materialized_postgresql_allow_automatic_update = 1"])
|
||||
assert_nested_table_is_created('postgresql_replica_0')
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number from numbers(10)")
|
||||
check_tables_are_synchronized('postgresql_replica_0');
|
||||
|
||||
# just check that it works, no check with `expected` becuase _version is taken as LSN, which will be different each time.
|
||||
result = instance.query('SELECT key, value, _sign, _version FROM test_database.postgresql_replica_0;')
|
||||
print(result)
|
||||
|
||||
cursor.execute("ALTER TABLE postgresql_replica_0 ADD COLUMN value2 integer")
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number, number from numbers(10, 10)")
|
||||
check_tables_are_synchronized('postgresql_replica_0');
|
||||
|
||||
result = instance.query('SELECT key, value, value2, _sign, _version FROM test_database.postgresql_replica_0;')
|
||||
print(result)
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica_0 SELECT number, number, number from numbers(20, 10)")
|
||||
check_tables_are_synchronized('postgresql_replica_0');
|
||||
|
||||
result = instance.query('SELECT key, value, value2, _sign, _version FROM test_database.postgresql_replica_0;')
|
||||
print(result)
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_multiple_databases(started_cluster):
|
||||
drop_materialized_db('test_database_1')
|
||||
drop_materialized_db('test_database_2')
|
||||
NUM_TABLES = 5
|
||||
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=False)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_db(cursor, 'postgres_database_1')
|
||||
create_postgres_db(cursor, 'postgres_database_2')
|
||||
|
||||
conn1 = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, database_name='postgres_database_1')
|
||||
conn2 = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, database_name='postgres_database_2')
|
||||
|
||||
cursor1 = conn1.cursor()
|
||||
cursor2 = conn2.cursor()
|
||||
|
||||
create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_1')
|
||||
create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_2')
|
||||
|
||||
cursors = [cursor1, cursor2]
|
||||
for cursor_id in range(len(cursors)):
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
create_postgres_table(cursors[cursor_id], table_name);
|
||||
instance.query("INSERT INTO postgres_database_{}.{} SELECT number, number from numbers(50)".format(cursor_id + 1, table_name))
|
||||
print('database 1 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_1';'''))
|
||||
print('database 2 tables: ', instance.query('''SELECT name FROM system.tables WHERE database = 'postgres_database_2';'''))
|
||||
|
||||
create_materialized_db(started_cluster.postgres_ip, started_cluster.postgres_port,
|
||||
'test_database_1', 'postgres_database_1')
|
||||
create_materialized_db(started_cluster.postgres_ip, started_cluster.postgres_port,
|
||||
'test_database_2', 'postgres_database_2')
|
||||
|
||||
cursors = [cursor1, cursor2]
|
||||
for cursor_id in range(len(cursors)):
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
instance.query("INSERT INTO postgres_database_{}.{} SELECT 50 + number, number from numbers(50)".format(cursor_id + 1, table_name))
|
||||
|
||||
for cursor_id in range(len(cursors)):
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
check_tables_are_synchronized(
|
||||
table_name, 'key', 'postgres_database_{}'.format(cursor_id + 1), 'test_database_{}'.format(cursor_id + 1));
|
||||
|
||||
drop_clickhouse_postgres_db('postgres_database_1')
|
||||
drop_clickhouse_postgres_db('postgres_database_2')
|
||||
drop_materialized_db('test_database_1')
|
||||
drop_materialized_db('test_database_2')
|
||||
|
||||
|
||||
def test_concurrent_transactions(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 6
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i));
|
||||
|
||||
def transaction(thread_id):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, auto_commit=False)
|
||||
cursor_ = conn.cursor()
|
||||
for query in queries:
|
||||
cursor_.execute(query.format(thread_id))
|
||||
print('thread {}, query {}'.format(thread_id, query))
|
||||
conn.commit()
|
||||
|
||||
threads = []
|
||||
threads_num = 6
|
||||
for i in range(threads_num):
|
||||
threads.append(threading.Thread(target=transaction, args=(i,)))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for thread in threads:
|
||||
time.sleep(random.uniform(0, 0.5))
|
||||
thread.start()
|
||||
for thread in threads:
|
||||
thread.join()
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
count1 = instance.query('SELECT count() FROM postgres_database.postgresql_replica_{}'.format(i))
|
||||
count2 = instance.query('SELECT count() FROM (SELECT * FROM test_database.postgresql_replica_{})'.format(i))
|
||||
print(int(count1), int(count2), sep=' ')
|
||||
assert(int(count1) == int(count2))
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_abrupt_connection_loss_while_heavy_replication(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 6
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i));
|
||||
|
||||
def transaction(thread_id):
|
||||
if thread_id % 2:
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, auto_commit=True)
|
||||
else:
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, auto_commit=False)
|
||||
cursor_ = conn.cursor()
|
||||
for query in queries:
|
||||
cursor_.execute(query.format(thread_id))
|
||||
print('thread {}, query {}'.format(thread_id, query))
|
||||
if thread_id % 2 == 0:
|
||||
conn.commit()
|
||||
|
||||
threads = []
|
||||
threads_num = 6
|
||||
for i in range(threads_num):
|
||||
threads.append(threading.Thread(target=transaction, args=(i,)))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for thread in threads:
|
||||
time.sleep(random.uniform(0, 0.5))
|
||||
thread.start()
|
||||
|
||||
# Join here because it takes time for data to reach wal
|
||||
for thread in threads:
|
||||
thread.join()
|
||||
time.sleep(1)
|
||||
started_cluster.pause_container('postgres1')
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i))
|
||||
print(result) # Just debug
|
||||
|
||||
started_cluster.unpause_container('postgres1')
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i))
|
||||
print(result) # Just debug
|
||||
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_drop_database_while_replication_startup_not_finished(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 5
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
create_postgres_table(cursor, table_name);
|
||||
instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(100000)".format(table_name))
|
||||
|
||||
for i in range(6):
|
||||
create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port)
|
||||
time.sleep(0.5 * i)
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_restart_server_while_replication_startup_not_finished(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 5
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
table_name = 'postgresql_replica_{}'.format(i)
|
||||
create_postgres_table(cursor, table_name);
|
||||
instance.query("INSERT INTO postgres_database.{} SELECT number, number from numbers(100000)".format(table_name))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port)
|
||||
time.sleep(0.5)
|
||||
instance.restart_clickhouse()
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_abrupt_server_restart_while_heavy_replication(started_cluster):
|
||||
drop_materialized_db()
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
NUM_TABLES = 6
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
create_postgres_table(cursor, 'postgresql_replica_{}'.format(i));
|
||||
|
||||
def transaction(thread_id):
|
||||
if thread_id % 2:
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, auto_commit=True)
|
||||
else:
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True, auto_commit=False)
|
||||
cursor_ = conn.cursor()
|
||||
for query in queries:
|
||||
cursor_.execute(query.format(thread_id))
|
||||
print('thread {}, query {}'.format(thread_id, query))
|
||||
if thread_id % 2 == 0:
|
||||
conn.commit()
|
||||
|
||||
threads = []
|
||||
threads_num = 6
|
||||
for i in range(threads_num):
|
||||
threads.append(threading.Thread(target=transaction, args=(i,)))
|
||||
|
||||
create_materialized_db(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for thread in threads:
|
||||
time.sleep(random.uniform(0, 0.5))
|
||||
thread.start()
|
||||
|
||||
# Join here because it takes time for data to reach wal
|
||||
for thread in threads:
|
||||
thread.join()
|
||||
instance.restart_clickhouse()
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i))
|
||||
print(result) # Just debug
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
check_tables_are_synchronized('postgresql_replica_{}'.format(i));
|
||||
|
||||
for i in range(NUM_TABLES):
|
||||
result = instance.query("SELECT count() FROM test_database.postgresql_replica_{}".format(i))
|
||||
print(result) # Just debug
|
||||
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
cluster.start()
|
||||
input("Cluster created, press any key to destroy...")
|
||||
cluster.shutdown()
|
@ -0,0 +1,11 @@
|
||||
<yandex>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
</yandex>
|
600
tests/integration/test_storage_postgresql_replica/test.py
Normal file
600
tests/integration/test_storage_postgresql_replica/test.py
Normal file
@ -0,0 +1,600 @@
|
||||
import pytest
|
||||
import time
|
||||
import psycopg2
|
||||
import os.path as p
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT
|
||||
from helpers.test_tools import TSV
|
||||
|
||||
import threading
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance', main_configs=['configs/log_conf.xml'], with_postgres=True, stay_alive=True)
|
||||
|
||||
postgres_table_template = """
|
||||
CREATE TABLE IF NOT EXISTS {} (
|
||||
key Integer NOT NULL, value Integer, PRIMARY KEY(key))
|
||||
"""
|
||||
|
||||
queries = [
|
||||
'INSERT INTO postgresql_replica select i, i from generate_series(0, 10000) as t(i);',
|
||||
'DELETE FROM postgresql_replica WHERE (value*value) % 3 = 0;',
|
||||
'UPDATE postgresql_replica SET value = value + 125 WHERE key % 2 = 0;',
|
||||
"UPDATE postgresql_replica SET key=key+20000 WHERE key%2=0",
|
||||
'INSERT INTO postgresql_replica select i, i from generate_series(40000, 50000) as t(i);',
|
||||
'DELETE FROM postgresql_replica WHERE key % 10 = 0;',
|
||||
'UPDATE postgresql_replica SET value = value + 101 WHERE key % 2 = 1;',
|
||||
"UPDATE postgresql_replica SET key=key+80000 WHERE key%2=1",
|
||||
'DELETE FROM postgresql_replica WHERE value % 2 = 0;',
|
||||
'UPDATE postgresql_replica SET value = value + 2000 WHERE key % 5 = 0;',
|
||||
'INSERT INTO postgresql_replica select i, i from generate_series(200000, 250000) as t(i);',
|
||||
'DELETE FROM postgresql_replica WHERE value % 3 = 0;',
|
||||
'UPDATE postgresql_replica SET value = value * 2 WHERE key % 3 = 0;',
|
||||
"UPDATE postgresql_replica SET key=key+500000 WHERE key%2=1",
|
||||
'INSERT INTO postgresql_replica select i, i from generate_series(1000000, 1050000) as t(i);',
|
||||
'DELETE FROM postgresql_replica WHERE value % 9 = 2;',
|
||||
"UPDATE postgresql_replica SET key=key+10000000",
|
||||
'UPDATE postgresql_replica SET value = value + 2 WHERE key % 3 = 1;',
|
||||
'DELETE FROM postgresql_replica WHERE value%5 = 0;'
|
||||
]
|
||||
|
||||
|
||||
@pytest.mark.timeout(30)
|
||||
def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database'):
|
||||
expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by))
|
||||
result = instance.query('select * from test.{} order by {};'.format(table_name, order_by))
|
||||
|
||||
while result != expected:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('select * from test.{} order by {};'.format(table_name, order_by))
|
||||
|
||||
assert(result == expected)
|
||||
|
||||
def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name='postgres_database'):
|
||||
if database == True:
|
||||
conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format(ip, port, database_name)
|
||||
else:
|
||||
conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(ip, port)
|
||||
|
||||
conn = psycopg2.connect(conn_string)
|
||||
if auto_commit:
|
||||
conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT)
|
||||
conn.autocommit = True
|
||||
return conn
|
||||
|
||||
def create_postgres_db(cursor, name):
|
||||
cursor.execute("CREATE DATABASE {}".format(name))
|
||||
|
||||
def create_clickhouse_postgres_db(ip, port, name='postgres_database'):
|
||||
instance.query('''
|
||||
CREATE DATABASE {}
|
||||
ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')'''.format(name, ip, port, name))
|
||||
|
||||
def create_materialized_table(ip, port):
|
||||
instance.query('''
|
||||
CREATE TABLE test.postgresql_replica (key UInt64, value UInt64)
|
||||
ENGINE = MaterializedPostgreSQL(
|
||||
'{}:{}', 'postgres_database', 'postgresql_replica', 'postgres', 'mysecretpassword')
|
||||
PRIMARY KEY key; '''.format(ip, port))
|
||||
|
||||
def create_postgres_table(cursor, table_name, replica_identity_full=False):
|
||||
cursor.execute("DROP TABLE IF EXISTS {}".format(table_name))
|
||||
cursor.execute(postgres_table_template.format(table_name))
|
||||
if replica_identity_full:
|
||||
cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name))
|
||||
|
||||
|
||||
def postgresql_replica_check_result(result, check=False, ref_file='test_postgresql_replica.reference'):
|
||||
fpath = p.join(p.dirname(__file__), ref_file)
|
||||
with open(fpath) as reference:
|
||||
if check:
|
||||
assert TSV(result) == TSV(reference)
|
||||
else:
|
||||
return TSV(result) == TSV(reference)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
conn = get_postgres_conn(ip=cluster.postgres_ip,
|
||||
port=cluster.postgres_port)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_db(cursor, 'postgres_database')
|
||||
create_clickhouse_postgres_db(ip=cluster.postgres_ip,
|
||||
port=cluster.postgres_port)
|
||||
|
||||
instance.query('CREATE DATABASE test')
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_initial_load_from_snapshot(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
while postgresql_replica_check_result(result) == False:
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_no_connection_at_startup(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
time.sleep(3)
|
||||
|
||||
instance.query('DETACH TABLE test.postgresql_replica')
|
||||
started_cluster.pause_container('postgres1')
|
||||
|
||||
instance.query('ATTACH TABLE test.postgresql_replica')
|
||||
time.sleep(3)
|
||||
started_cluster.unpause_container('postgres1')
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while int(result) == 0:
|
||||
time.sleep(0.5);
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_detach_attach_is_ok(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) == 0):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
instance.query('DETACH TABLE test.postgresql_replica')
|
||||
instance.query('ATTACH TABLE test.postgresql_replica')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
while postgresql_replica_check_result(result) == False:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_replicating_insert_queries(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 10):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)")
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)")
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 30):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)")
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)")
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 50):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_replicating_delete_queries(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
while postgresql_replica_check_result(result) == False:
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)")
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while int(result) != 100:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
cursor.execute('DELETE FROM postgresql_replica WHERE key > 49;')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
while postgresql_replica_check_result(result) == False:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_replicating_update_queries(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 50):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
cursor.execute('UPDATE postgresql_replica SET value = value - 10;')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
while postgresql_replica_check_result(result) == False:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_resume_from_written_version(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 50):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)")
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 100):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
instance.query('DETACH TABLE test.postgresql_replica')
|
||||
|
||||
cursor.execute('DELETE FROM postgresql_replica WHERE key > 49;')
|
||||
cursor.execute('UPDATE postgresql_replica SET value = value - 10;')
|
||||
|
||||
instance.query('ATTACH TABLE test.postgresql_replica')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
while postgresql_replica_check_result(result) == False:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica ORDER BY key;')
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_many_replication_messages(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 100000):
|
||||
time.sleep(0.2)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
print("SYNC OK")
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000, 100000)")
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while (int(result) != 200000):
|
||||
time.sleep(1)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
print("INSERT OK")
|
||||
|
||||
result = instance.query('SELECT key FROM test.postgresql_replica ORDER BY key;')
|
||||
expected = instance.query("SELECT number from numbers(200000)")
|
||||
assert(result == expected)
|
||||
|
||||
cursor.execute('UPDATE postgresql_replica SET value = key + 1 WHERE key < 100000;')
|
||||
|
||||
result = instance.query('SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;')
|
||||
expected = instance.query("SELECT number from numbers(100000)")
|
||||
|
||||
while (result != expected):
|
||||
time.sleep(1)
|
||||
result = instance.query('SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;')
|
||||
print("UPDATE OK")
|
||||
|
||||
cursor.execute('DELETE FROM postgresql_replica WHERE key % 2 = 1;')
|
||||
cursor.execute('DELETE FROM postgresql_replica WHERE key != value;')
|
||||
|
||||
result = instance.query('SELECT count() FROM (SELECT * FROM test.postgresql_replica);')
|
||||
while (int(result) != 50000):
|
||||
time.sleep(1)
|
||||
result = instance.query('SELECT count() FROM (SELECT * FROM test.postgresql_replica);')
|
||||
print("DELETE OK")
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_connection_loss(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
i = 50
|
||||
while i < 100000:
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format(i))
|
||||
i += 10000
|
||||
|
||||
started_cluster.pause_container('postgres1')
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
print(int(result))
|
||||
time.sleep(6)
|
||||
|
||||
started_cluster.unpause_container('postgres1')
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while int(result) < 100050:
|
||||
time.sleep(1)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
assert(int(result) == 100050)
|
||||
|
||||
|
||||
@pytest.mark.timeout(320)
|
||||
def test_clickhouse_restart(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
i = 50
|
||||
while i < 100000:
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format(i))
|
||||
i += 10000
|
||||
|
||||
instance.restart_clickhouse()
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while int(result) < 100050:
|
||||
time.sleep(1)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
print(result)
|
||||
assert(int(result) == 100050)
|
||||
|
||||
|
||||
def test_rename_table(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25)")
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while int(result) != 25:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
instance.query('RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed')
|
||||
assert(int(instance.query('SELECT count() FROM test.postgresql_replica_renamed;')) == 25)
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25, 25)")
|
||||
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;')
|
||||
while int(result) != 50:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica_renamed;')
|
||||
|
||||
result = instance.query('SELECT * FROM test.postgresql_replica_renamed ORDER BY key;')
|
||||
postgresql_replica_check_result(result, True)
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica_renamed')
|
||||
|
||||
|
||||
def test_virtual_columns(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)")
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
while int(result) != 10:
|
||||
time.sleep(0.5)
|
||||
result = instance.query('SELECT count() FROM test.postgresql_replica;')
|
||||
|
||||
# just check that it works, no check with `expected` becuase _version is taken as LSN, which will be different each time.
|
||||
result = instance.query('SELECT key, value, _sign, _version FROM test.postgresql_replica;')
|
||||
print(result)
|
||||
cursor.execute('DROP TABLE postgresql_replica;')
|
||||
|
||||
|
||||
def test_abrupt_connection_loss_while_heavy_replication(started_cluster):
|
||||
instance.query("DROP DATABASE IF EXISTS test_database")
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for i in range(len(queries)):
|
||||
query = queries[i]
|
||||
cursor.execute(query)
|
||||
print('query {}'.format(query))
|
||||
|
||||
started_cluster.pause_container('postgres1')
|
||||
|
||||
result = instance.query("SELECT count() FROM test.postgresql_replica")
|
||||
print(result) # Just debug
|
||||
|
||||
started_cluster.unpause_container('postgres1')
|
||||
|
||||
check_tables_are_synchronized('postgresql_replica');
|
||||
|
||||
result = instance.query("SELECT count() FROM test.postgresql_replica")
|
||||
print(result) # Just debug
|
||||
|
||||
|
||||
def test_abrupt_server_restart_while_heavy_replication(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port)
|
||||
|
||||
for query in queries:
|
||||
cursor.execute(query)
|
||||
print('query {}'.format(query))
|
||||
|
||||
instance.restart_clickhouse()
|
||||
|
||||
result = instance.query("SELECT count() FROM test.postgresql_replica")
|
||||
print(result) # Just debug
|
||||
|
||||
check_tables_are_synchronized('postgresql_replica');
|
||||
|
||||
result = instance.query("SELECT count() FROM test.postgresql_replica")
|
||||
print(result) # Just debug
|
||||
|
||||
|
||||
def test_drop_table_immediately(started_cluster):
|
||||
conn = get_postgres_conn(ip=started_cluster.postgres_ip,
|
||||
port=started_cluster.postgres_port,
|
||||
database=True)
|
||||
cursor = conn.cursor()
|
||||
create_postgres_table(cursor, 'postgresql_replica');
|
||||
instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)")
|
||||
|
||||
instance.query('DROP TABLE IF EXISTS test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port)
|
||||
instance.query('DROP TABLE test.postgresql_replica')
|
||||
create_materialized_table(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port)
|
||||
check_tables_are_synchronized('postgresql_replica');
|
||||
instance.query('DROP TABLE test.postgresql_replica')
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
cluster.start()
|
||||
input("Cluster created, press any key to destroy...")
|
||||
cluster.shutdown()
|
@ -0,0 +1,50 @@
|
||||
0 0
|
||||
1 1
|
||||
2 2
|
||||
3 3
|
||||
4 4
|
||||
5 5
|
||||
6 6
|
||||
7 7
|
||||
8 8
|
||||
9 9
|
||||
10 10
|
||||
11 11
|
||||
12 12
|
||||
13 13
|
||||
14 14
|
||||
15 15
|
||||
16 16
|
||||
17 17
|
||||
18 18
|
||||
19 19
|
||||
20 20
|
||||
21 21
|
||||
22 22
|
||||
23 23
|
||||
24 24
|
||||
25 25
|
||||
26 26
|
||||
27 27
|
||||
28 28
|
||||
29 29
|
||||
30 30
|
||||
31 31
|
||||
32 32
|
||||
33 33
|
||||
34 34
|
||||
35 35
|
||||
36 36
|
||||
37 37
|
||||
38 38
|
||||
39 39
|
||||
40 40
|
||||
41 41
|
||||
42 42
|
||||
43 43
|
||||
44 44
|
||||
45 45
|
||||
46 46
|
||||
47 47
|
||||
48 48
|
||||
49 49
|
Loading…
Reference in New Issue
Block a user