mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Fix style & compilation.
This commit is contained in:
parent
5127eeee7b
commit
000b184691
@ -1,3 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/IBackupCoordination.h>
|
||||
#include <Common/ZooKeeper/Common.h>
|
||||
#include <map>
|
||||
|
@ -1,5 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Backups/IBackupCoordination.h>
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Backups/BackupInfo.h>
|
||||
#include <Backups/Common/BackupSettings.h>
|
||||
#include <Backups/BackupSettings.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Parsers/ASTBackupQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
@ -1,6 +1,6 @@
|
||||
#include <Backups/BackupUtils.h>
|
||||
#include <Backups/BackupEntryFromMemory.h>
|
||||
#include <Backups/Common/BackupSettings.h>
|
||||
#include <Backups/BackupSettings.h>
|
||||
#include <Backups/DDLCompareUtils.h>
|
||||
#include <Backups/DDLRenamingVisitor.h>
|
||||
#include <Backups/IBackup.h>
|
||||
|
@ -25,7 +25,7 @@ size_t BackupsWorker::add(const String & backup_name, BackupStatus status, const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
|
||||
UInt64 task_id = ++current_task_id;
|
||||
size_t task_id = ++current_task_id;
|
||||
size_t pos;
|
||||
auto it = entries_by_name.find(backup_name);
|
||||
if (it != entries_by_name.end())
|
||||
@ -51,7 +51,7 @@ size_t BackupsWorker::add(const String & backup_name, BackupStatus status, const
|
||||
return task_id;
|
||||
}
|
||||
|
||||
void BackupsWorker::update(UInt64 task_id, BackupStatus status, const String & error)
|
||||
void BackupsWorker::update(size_t task_id, BackupStatus status, const String & error)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = entries_by_task_id.find(task_id);
|
||||
@ -63,7 +63,7 @@ void BackupsWorker::update(UInt64 task_id, BackupStatus status, const String & e
|
||||
entry.timestamp = std::time(nullptr);
|
||||
}
|
||||
|
||||
BackupsWorker::Entry BackupsWorker::getEntry(UInt64 task_id) const
|
||||
BackupsWorker::Entry BackupsWorker::getEntry(size_t task_id) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = entries_by_task_id.find(task_id);
|
||||
|
@ -14,22 +14,22 @@ class BackupsWorker
|
||||
public:
|
||||
static BackupsWorker & instance();
|
||||
|
||||
UInt64 add(const String & backup_name, BackupStatus status, const String & error = {});
|
||||
void update(UInt64 task_id, BackupStatus status, const String & error = {});
|
||||
size_t add(const String & backup_name, BackupStatus status, const String & error = {});
|
||||
void update(size_t task_id, BackupStatus status, const String & error = {});
|
||||
|
||||
struct Entry
|
||||
{
|
||||
String backup_name;
|
||||
UInt64 task_id;
|
||||
size_t task_id;
|
||||
BackupStatus status;
|
||||
String error;
|
||||
time_t timestamp;
|
||||
};
|
||||
|
||||
Entry getEntry(UInt64 task_id) const;
|
||||
Entry getEntry(size_t task_id) const;
|
||||
std::vector<Entry> getEntries() const;
|
||||
|
||||
/// Schedules a new task and perfoms it in the background thread.
|
||||
/// Schedules a new task and performs it in the background thread.
|
||||
void run(std::function<void()> && task);
|
||||
|
||||
/// Waits until all tasks have been completed.
|
||||
@ -40,9 +40,9 @@ private:
|
||||
|
||||
mutable std::mutex mutex;
|
||||
std::vector<Entry> entries;
|
||||
std::unordered_map<String, UInt64> entries_by_name;
|
||||
std::unordered_map<UInt64, size_t> entries_by_task_id;
|
||||
UInt64 current_task_id = 0;
|
||||
std::unordered_map<String, size_t /* position in entries */> entries_by_name;
|
||||
std::unordered_map<size_t /* task_id */, size_t /* position in entries */ > entries_by_task_id;
|
||||
size_t current_task_id = 0;
|
||||
ThreadPool thread_pool;
|
||||
};
|
||||
|
||||
|
@ -1,5 +0,0 @@
|
||||
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
|
||||
|
||||
add_headers_and_sources(clickhouse_common_backups .)
|
||||
add_library(clickhouse_common_backups ${clickhouse_common_backups_headers} ${clickhouse_common_backups_sources})
|
||||
target_link_libraries(clickhouse_common_backups PUBLIC clickhouse_common_io)
|
@ -1,59 +0,0 @@
|
||||
#include <Backups/Common/rewriteBackupQueryWithoutOnCluster.h>
|
||||
#include <Backups/Common/BackupSettings.h>
|
||||
#include <Backups/Common/RestoreSettings.h>
|
||||
#include <Parsers/ASTBackupQuery.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
void setDatabaseInElements(ASTBackupQuery::Elements & elements, const String & new_database)
|
||||
{
|
||||
for (auto & element : elements)
|
||||
{
|
||||
if (element.type == ASTBackupQuery::TABLE)
|
||||
{
|
||||
if (element.name.first.empty() && !element.name.second.empty() && !element.name_is_in_temp_db)
|
||||
element.name.first = new_database;
|
||||
if (element.new_name.first.empty() && !element.name.second.empty() && !element.name_is_in_temp_db)
|
||||
element.new_name.first = new_database;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::shared_ptr<ASTBackupQuery>
|
||||
rewriteBackupQueryWithoutOnCluster(const ASTBackupQuery & backup_query, const WithoutOnClusterASTRewriteParams & params)
|
||||
{
|
||||
auto backup_settings = BackupSettings::fromBackupQuery(backup_query);
|
||||
backup_settings.internal = true;
|
||||
backup_settings.async = false;
|
||||
backup_settings.shard_num = params.shard_index;
|
||||
backup_settings.replica_num = params.replica_index;
|
||||
auto new_query = std::static_pointer_cast<ASTBackupQuery>(backup_query.clone());
|
||||
new_query->cluster.clear();
|
||||
backup_settings.copySettingsToBackupQuery(*new_query);
|
||||
setDatabaseInElements(new_query->elements, params.default_database);
|
||||
return new_query;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<ASTBackupQuery>
|
||||
rewriteRestoreQueryWithoutOnCluster(const ASTBackupQuery & restore_query, const WithoutOnClusterASTRewriteParams & params)
|
||||
{
|
||||
auto restore_settings = RestoreSettings::fromRestoreQuery(restore_query);
|
||||
restore_settings.internal = true;
|
||||
restore_settings.async = false;
|
||||
restore_settings.shard_num = params.shard_index;
|
||||
restore_settings.replica_num = params.replica_index;
|
||||
auto new_query = std::static_pointer_cast<ASTBackupQuery>(restore_query.clone());
|
||||
new_query->cluster.clear();
|
||||
restore_settings.copySettingsToRestoreQuery(*new_query);
|
||||
setDatabaseInElements(new_query->elements, params.default_database);
|
||||
return new_query;
|
||||
}
|
||||
|
||||
}
|
@ -1,19 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ASTBackupQuery;
|
||||
struct WithoutOnClusterASTRewriteParams;
|
||||
|
||||
/// Rewrites elements of BACKUP-ON-CLUSTER query after receiving it on shards or replica.
|
||||
std::shared_ptr<ASTBackupQuery>
|
||||
rewriteBackupQueryWithoutOnCluster(const ASTBackupQuery & backup_query, const WithoutOnClusterASTRewriteParams & params);
|
||||
|
||||
/// Rewrites elements of RESTORE-ON-CLUSTER query after receiving it on shards or replica.
|
||||
std::shared_ptr<ASTBackupQuery>
|
||||
rewriteRestoreQueryWithoutOnCluster(const ASTBackupQuery & restore_query, const WithoutOnClusterASTRewriteParams & params);
|
||||
|
||||
}
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <optional>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Backups/IRestoreCoordination.h>
|
||||
#include <Common/ZooKeeper/Common.h>
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Backups/IRestoreCoordination.h>
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Backups/BackupInfo.h>
|
||||
#include <Backups/Common/RestoreSettings.h>
|
||||
#include <Backups/RestoreSettings.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Core/SettingsFields.h>
|
||||
#include <Parsers/ASTBackupQuery.h>
|
@ -1,6 +1,6 @@
|
||||
#include <Backups/RestoreUtils.h>
|
||||
#include <Backups/BackupUtils.h>
|
||||
#include <Backups/Common/RestoreSettings.h>
|
||||
#include <Backups/RestoreSettings.h>
|
||||
#include <Backups/DDLCompareUtils.h>
|
||||
#include <Backups/DDLRenamingVisitor.h>
|
||||
#include <Backups/IBackup.h>
|
||||
|
@ -215,7 +215,6 @@ if (TARGET ch_contrib::jemalloc)
|
||||
endif()
|
||||
|
||||
add_subdirectory(Access/Common)
|
||||
add_subdirectory(Backups/Common)
|
||||
add_subdirectory(Common/ZooKeeper)
|
||||
add_subdirectory(Common/Config)
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Interpreters/InterpreterBackupQuery.h>
|
||||
#include <Backups/Common/BackupSettings.h>
|
||||
#include <Backups/Common/RestoreSettings.h>
|
||||
#include <Backups/BackupSettings.h>
|
||||
#include <Backups/RestoreSettings.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/IBackupEntry.h>
|
||||
#include <Backups/IRestoreTask.h>
|
||||
@ -51,7 +51,7 @@ namespace
|
||||
return BackupFactory::instance().createBackup(params);
|
||||
}
|
||||
|
||||
void executeBackupSync(const ASTBackupQuery & query, UInt64 task_id, const ContextPtr & context, const BackupInfo & backup_info, const BackupSettings & backup_settings, bool no_throw = false)
|
||||
void executeBackupSync(const ASTBackupQuery & query, size_t task_id, const ContextPtr & context, const BackupInfo & backup_info, const BackupSettings & backup_settings, bool no_throw = false)
|
||||
{
|
||||
auto & worker = BackupsWorker::instance();
|
||||
bool is_internal_backup = backup_settings.internal;
|
||||
@ -107,7 +107,7 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void executeRestoreSync(const ASTBackupQuery & query, UInt64 task_id, ContextMutablePtr context, const BackupInfo & backup_info, const RestoreSettings & restore_settings, bool no_throw = false)
|
||||
void executeRestoreSync(const ASTBackupQuery & query, size_t task_id, ContextMutablePtr context, const BackupInfo & backup_info, const RestoreSettings & restore_settings, bool no_throw = false)
|
||||
{
|
||||
auto & worker = BackupsWorker::instance();
|
||||
bool is_internal_restore = restore_settings.internal;
|
||||
@ -155,7 +155,7 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
UInt64 executeBackup(const ASTBackupQuery & query, const ContextPtr & context)
|
||||
size_t executeBackup(const ASTBackupQuery & query, const ContextPtr & context)
|
||||
{
|
||||
const auto backup_info = BackupInfo::fromAST(*query.backup_name);
|
||||
const auto backup_settings = BackupSettings::fromBackupQuery(query);
|
||||
@ -175,7 +175,7 @@ namespace
|
||||
return task_id;
|
||||
}
|
||||
|
||||
UInt64 executeRestore(const ASTBackupQuery & query, ContextMutablePtr context)
|
||||
size_t executeRestore(const ASTBackupQuery & query, ContextMutablePtr context)
|
||||
{
|
||||
const auto backup_info = BackupInfo::fromAST(*query.backup_name);
|
||||
const auto restore_settings = RestoreSettings::fromRestoreQuery(query);
|
||||
@ -195,7 +195,7 @@ namespace
|
||||
return task_id;
|
||||
}
|
||||
|
||||
Block getResultRow(UInt64 task_id)
|
||||
Block getResultRow(size_t task_id)
|
||||
{
|
||||
if (!task_id)
|
||||
return {};
|
||||
@ -223,7 +223,7 @@ BlockIO InterpreterBackupQuery::execute()
|
||||
{
|
||||
const auto & query = query_ptr->as<const ASTBackupQuery &>();
|
||||
|
||||
UInt64 task_id;
|
||||
size_t task_id;
|
||||
if (query.kind == ASTBackupQuery::BACKUP)
|
||||
task_id = executeBackup(query, context);
|
||||
else
|
||||
|
@ -1,7 +1,9 @@
|
||||
#include <Parsers/ASTBackupQuery.h>
|
||||
#include <Backups/Common/rewriteBackupQueryWithoutOnCluster.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -137,8 +139,49 @@ namespace
|
||||
settings->format(format);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void setDatabaseInElements(ASTBackupQuery::Elements & elements, const String & new_database)
|
||||
{
|
||||
for (auto & element : elements)
|
||||
{
|
||||
if (element.type == ASTBackupQuery::TABLE)
|
||||
{
|
||||
if (element.name.first.empty() && !element.name.second.empty() && !element.name_is_in_temp_db)
|
||||
element.name.first = new_database;
|
||||
if (element.new_name.first.empty() && !element.name.second.empty() && !element.name_is_in_temp_db)
|
||||
element.new_name.first = new_database;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ASTPtr rewriteSettingsWithoutOnCluster(ASTPtr settings, const WithoutOnClusterASTRewriteParams & params)
|
||||
{
|
||||
SettingsChanges changes;
|
||||
if (settings)
|
||||
changes = assert_cast<ASTSetQuery *>(settings.get())->changes;
|
||||
|
||||
boost::remove_erase_if(
|
||||
changes,
|
||||
[](const SettingChange & change)
|
||||
{
|
||||
const String & name = change.name;
|
||||
return (name == "internal") || (name == "async") || (name == "shard_num") || (name == "replica_num");
|
||||
});
|
||||
|
||||
changes.emplace_back("internal", true);
|
||||
changes.emplace_back("async", false);
|
||||
changes.emplace_back("shard_num", params.shard_index);
|
||||
changes.emplace_back("replica_num", params.replica_index);
|
||||
|
||||
auto out_settings = std::shared_ptr<ASTSetQuery>();
|
||||
out_settings->changes = std::move(changes);
|
||||
out_settings->is_standalone = false;
|
||||
return out_settings;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
String ASTBackupQuery::getID(char) const
|
||||
{
|
||||
return (kind == Kind::BACKUP) ? "BackupQuery" : "RestoreQuery";
|
||||
@ -168,10 +211,11 @@ void ASTBackupQuery::formatImpl(const FormatSettings & format, FormatState &, Fo
|
||||
|
||||
ASTPtr ASTBackupQuery::getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const
|
||||
{
|
||||
if (kind == ASTBackupQuery::Kind::BACKUP)
|
||||
return rewriteBackupQueryWithoutOnCluster(*this, params);
|
||||
else
|
||||
return rewriteRestoreQueryWithoutOnCluster(*this, params);
|
||||
auto new_query = std::static_pointer_cast<ASTBackupQuery>(clone());
|
||||
new_query->cluster.clear();
|
||||
new_query->settings = rewriteSettingsWithoutOnCluster(new_query->settings, params);
|
||||
setDatabaseInElements(new_query->elements, params.default_database);
|
||||
return new_query;
|
||||
}
|
||||
|
||||
|
||||
|
@ -5,7 +5,6 @@ add_headers_and_sources(clickhouse_parsers ./Access)
|
||||
add_headers_and_sources(clickhouse_parsers ./MySQL)
|
||||
add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources})
|
||||
target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access)
|
||||
target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_backups)
|
||||
|
||||
if (USE_DEBUG_HELPERS)
|
||||
# CMake generator expression will do insane quoting when it encounters special character like quotes, spaces, etc.
|
||||
|
Loading…
Reference in New Issue
Block a user