ClickHouse/dbms/src/Databases/DatabaseMySQL.h

125 lines
4.1 KiB
C++
Raw Normal View History

#pragma once
#include "config_core.h"
2019-06-13 08:19:26 +00:00
#if USE_MYSQL
#include <mysqlxx/Pool.h>
#include <Databases/DatabasesCommon.h>
namespace DB
{
/** Real-time access to table list and table structure from remote MySQL
* It doesn't make any manipulations with filesystem.
* All tables are created by calling code after real-time pull-out structure from remote MySQL
*/
class DatabaseMySQL : public IDatabase
{
public:
~DatabaseMySQL() override;
DatabaseMySQL(const Context & context_, const String & database_name_, const String & mysql_host_name_, const UInt16 & mysql_port_,
const String & mysql_database_name_, const String & mysql_user_name_, const String & mysql_user_password_);
String getEngineName() const override { return "MySQL"; }
String getDatabaseName() const override { return database_name; }
2019-06-14 16:18:48 +00:00
bool empty(const Context & context) const override;
2019-06-15 23:44:51 +00:00
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
2019-06-14 16:18:48 +00:00
bool isTableExist(const Context & context, const String & name) const override;
2019-06-14 16:18:48 +00:00
StoragePtr tryGetTable(const Context & context, const String & name) const override;
2019-06-14 16:18:48 +00:00
ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const override;
time_t getTableMetadataModificationTime(const Context & context, const String & name) override;
void shutdown() override;
StoragePtr detachTable(const String &) override
{
throw Exception("MySQL database engine does not support detach table.", ErrorCodes::NOT_IMPLEMENTED);
}
void loadTables(Context &, ThreadPool *, bool) override
{
/// do nothing
}
void removeTable(const Context &, const String &) override
{
throw Exception("MySQL database engine does not support remove table.", ErrorCodes::NOT_IMPLEMENTED);
}
void attachTable(const String &, const StoragePtr &) override
{
throw Exception("MySQL database engine does not support attach table.", ErrorCodes::NOT_IMPLEMENTED);
}
void renameTable(const Context &, const String &, IDatabase &, const String &) override
{
throw Exception("MySQL database engine does not support rename table.", ErrorCodes::NOT_IMPLEMENTED);
}
void createTable(const Context &, const String &, const StoragePtr &, const ASTPtr &) override
{
throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED);
}
void alterTable(const Context &, const String &, const ColumnsDescription &, const IndicesDescription &, const ASTModifier &) override
{
throw Exception("MySQL database engine does not support alter table.", ErrorCodes::NOT_IMPLEMENTED);
}
private:
struct MySQLStorageInfo
{
StoragePtr storage;
UInt64 modification_time;
ASTPtr create_table_query;
};
const Context global_context;
const String database_name;
const String mysql_host_name;
const UInt16 mysql_port;
const String mysql_database_name;
const String mysql_user_name;
const String mysql_user_password;
2019-06-14 16:18:48 +00:00
mutable std::mutex mutex;
std::atomic<bool> quit{false};
std::condition_variable cond;
2019-06-14 16:18:48 +00:00
mutable mysqlxx::Pool mysql_pool;
mutable std::vector<StoragePtr> outdated_tables;
mutable std::map<String, MySQLStorageInfo> local_tables_cache;
void cleanOutdatedTables();
2019-06-14 16:18:48 +00:00
void fetchTablesIntoLocalCache() const;
2019-06-14 16:18:48 +00:00
std::map<String, UInt64> fetchTablesWithModificationTime() const;
DatabaseMySQL::MySQLStorageInfo createStorageInfo(
2019-06-14 16:18:48 +00:00
const String & table_name, const NamesAndTypesList & columns_name_and_type, const UInt64 & table_modification_time) const;
2019-06-14 16:18:48 +00:00
std::map<String, NamesAndTypesList> fetchTablesColumnsList(const std::vector<String> & tables_name) const;
2019-06-14 16:18:48 +00:00
void destroyLocalCacheExtraTables(const std::map<String, UInt64> & tables_with_modification_time) const;
2019-06-14 16:18:48 +00:00
void fetchLatestTablesStructureIntoCache(const std::map<String, UInt64> & tables_modification_time) const;
ThreadFromGlobalPool thread{&DatabaseMySQL::cleanOutdatedTables, this};
};
}
2019-06-13 08:19:26 +00:00
#endif