mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
try fix backward compatibility
This commit is contained in:
parent
8cba35bbab
commit
80e082d3e2
@ -268,10 +268,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
|||||||
|
|
||||||
global_context->setPath(path);
|
global_context->setPath(path);
|
||||||
|
|
||||||
/// Create directories for 'path' and for default database, if not exist.
|
|
||||||
Poco::File(path + "data/" + default_database).createDirectories();
|
|
||||||
Poco::File(path + "metadata/" + default_database).createDirectories();
|
|
||||||
|
|
||||||
/// Check that we have read and write access to all data paths
|
/// Check that we have read and write access to all data paths
|
||||||
auto disk_selector = global_context->getDiskSelector();
|
auto disk_selector = global_context->getDiskSelector();
|
||||||
for (const auto & [name, disk] : disk_selector.getDisksMap())
|
for (const auto & [name, disk] : disk_selector.getDisksMap())
|
||||||
@ -529,7 +525,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
|||||||
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
|
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
|
||||||
attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper);
|
attachSystemTablesServer(*global_context->getDatabase("system"), has_zookeeper);
|
||||||
/// Then, load remaining databases
|
/// Then, load remaining databases
|
||||||
loadMetadata(*global_context);
|
loadMetadata(*global_context, default_database);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
|
@ -17,6 +17,7 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, const Contex
|
|||||||
: DatabaseOrdinary(name_, metadata_path_, context_)
|
: DatabaseOrdinary(name_, metadata_path_, context_)
|
||||||
{
|
{
|
||||||
data_path = "store/";
|
data_path = "store/";
|
||||||
|
log = &Logger::get("DatabaseAtomic (" + name_ + ")");
|
||||||
}
|
}
|
||||||
|
|
||||||
String DatabaseAtomic::getDataPath(const String & table_name) const
|
String DatabaseAtomic::getDataPath(const String & table_name) const
|
||||||
|
@ -30,10 +30,9 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
|
|
||||||
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_)
|
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_)
|
||||||
: DatabaseOnDisk(name_, metadata_path_, "DatabaseLazy (" + name_ + ")")
|
: DatabaseOnDisk(name_, metadata_path_, "DatabaseLazy (" + name_ + ")", context_)
|
||||||
, expiration_time(expiration_time_)
|
, expiration_time(expiration_time_)
|
||||||
{
|
{
|
||||||
Poco::File(context_.getPath() + getDataPath()).createDirectories();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -142,6 +142,16 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
|||||||
return statement_stream.str();
|
return statement_stream.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context_)
|
||||||
|
: DatabaseWithOwnTablesBase(name, logger)
|
||||||
|
, metadata_path(metadata_path_)
|
||||||
|
, data_path("data/" + escapeForFileName(database_name) + "/")
|
||||||
|
{
|
||||||
|
Poco::File(context_.getPath() + getDataPath()).createDirectories();
|
||||||
|
Poco::File(getMetadataPath()).createDirectories();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void DatabaseOnDisk::createTable(
|
void DatabaseOnDisk::createTable(
|
||||||
const Context & context,
|
const Context & context,
|
||||||
const String & table_name,
|
const String & table_name,
|
||||||
|
@ -32,10 +32,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query);
|
|||||||
class DatabaseOnDisk : public DatabaseWithOwnTablesBase
|
class DatabaseOnDisk : public DatabaseWithOwnTablesBase
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger)
|
DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger, const Context & context_);
|
||||||
: DatabaseWithOwnTablesBase(name, logger)
|
|
||||||
, metadata_path(metadata_path_)
|
|
||||||
, data_path("data/" + escapeForFileName(database_name) + "/") {}
|
|
||||||
|
|
||||||
void createTable(
|
void createTable(
|
||||||
const Context & context,
|
const Context & context,
|
||||||
|
@ -93,9 +93,8 @@ void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, Atomic
|
|||||||
|
|
||||||
|
|
||||||
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_)
|
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_)
|
||||||
: DatabaseWithDictionaries(name_, metadata_path_,"DatabaseOrdinary (" + name_ + ")")
|
: DatabaseWithDictionaries(name_, metadata_path_,"DatabaseOrdinary (" + name_ + ")", context_)
|
||||||
{
|
{
|
||||||
Poco::File(context_.getPath() + getDataPath()).createDirectories();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -26,8 +26,8 @@ public:
|
|||||||
bool isDictionaryExist(const Context & context, const String & dictionary_name) const override;
|
bool isDictionaryExist(const Context & context, const String & dictionary_name) const override;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & logger)
|
DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & logger, const Context & context_)
|
||||||
: DatabaseOnDisk(name, metadata_path_, logger) {}
|
: DatabaseOnDisk(name, metadata_path_, logger, context_) {}
|
||||||
|
|
||||||
StoragePtr getDictionaryStorage(const Context & context, const String & table_name) const;
|
StoragePtr getDictionaryStorage(const Context & context, const String & table_name) const;
|
||||||
|
|
||||||
|
@ -101,9 +101,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
|||||||
{
|
{
|
||||||
/// For new-style databases engine is explicitly specified in .sql
|
/// For new-style databases engine is explicitly specified in .sql
|
||||||
/// When attaching old-style database during server startup, we must always use Ordinary engine
|
/// When attaching old-style database during server startup, we must always use Ordinary engine
|
||||||
// FIXME maybe throw an exception if it's an ATTACH DATABASE query from user (it's not server startup) and engine is not specified
|
//FIXME is it possible, that database engine is not specified in metadata file?
|
||||||
bool old_style_database = create.attach ||
|
if (create.attach)
|
||||||
context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary;
|
throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||||
|
bool old_style_database = context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary;
|
||||||
auto engine = std::make_shared<ASTFunction>();
|
auto engine = std::make_shared<ASTFunction>();
|
||||||
auto storage = std::make_shared<ASTStorage>();
|
auto storage = std::make_shared<ASTStorage>();
|
||||||
engine->name = old_style_database ? "Ordinary" : "Atomic";
|
engine->name = old_style_database ? "Ordinary" : "Atomic";
|
||||||
@ -126,11 +127,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
|||||||
|
|
||||||
|
|
||||||
String database_name_escaped = escapeForFileName(database_name);
|
String database_name_escaped = escapeForFileName(database_name);
|
||||||
|
|
||||||
/// Create directories for tables metadata.
|
|
||||||
String path = context.getPath();
|
String path = context.getPath();
|
||||||
String metadata_path = path + "metadata/" + database_name_escaped + "/";
|
String metadata_path = path + "metadata/" + database_name_escaped + "/";
|
||||||
Poco::File(metadata_path).createDirectory();
|
|
||||||
|
|
||||||
DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, context);
|
DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, context);
|
||||||
|
|
||||||
|
@ -1,11 +1,6 @@
|
|||||||
#include <iomanip>
|
|
||||||
#include <thread>
|
|
||||||
#include <future>
|
|
||||||
|
|
||||||
#include <Common/ThreadPool.h>
|
#include <Common/ThreadPool.h>
|
||||||
|
|
||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
#include <Poco/FileStream.h>
|
|
||||||
|
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
#include <Parsers/ASTCreateQuery.h>
|
||||||
@ -21,7 +16,6 @@
|
|||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <Common/escapeForFileName.h>
|
#include <Common/escapeForFileName.h>
|
||||||
|
|
||||||
#include <Common/Stopwatch.h>
|
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
|
||||||
@ -39,7 +33,6 @@ static void executeCreateQuery(
|
|||||||
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0);
|
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0);
|
||||||
|
|
||||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||||
ast_create_query.attach = true;
|
|
||||||
ast_create_query.database = database;
|
ast_create_query.database = database;
|
||||||
|
|
||||||
InterpreterCreateQuery interpreter(ast, context);
|
InterpreterCreateQuery interpreter(ast, context);
|
||||||
@ -55,20 +48,27 @@ static void loadDatabase(
|
|||||||
const String & database_path,
|
const String & database_path,
|
||||||
bool force_restore_data)
|
bool force_restore_data)
|
||||||
{
|
{
|
||||||
/// There may exist .sql file with database creation statement.
|
|
||||||
/// Or, if it is absent, then database with Ordinary engine is created.
|
|
||||||
|
|
||||||
String database_attach_query;
|
String database_attach_query;
|
||||||
String database_metadata_file = database_path + ".sql";
|
String database_metadata_file = database_path + ".sql";
|
||||||
|
|
||||||
if (Poco::File(database_metadata_file).exists())
|
if (Poco::File(database_metadata_file).exists())
|
||||||
{
|
{
|
||||||
|
/// There are .sql file with database creation statement.
|
||||||
ReadBufferFromFile in(database_metadata_file, 1024);
|
ReadBufferFromFile in(database_metadata_file, 1024);
|
||||||
readStringUntilEOF(database_attach_query, in);
|
readStringUntilEOF(database_attach_query, in);
|
||||||
}
|
}
|
||||||
|
else if (Poco::File(database_path).exists())
|
||||||
|
{
|
||||||
|
/// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default)
|
||||||
|
database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary";
|
||||||
|
}
|
||||||
else
|
else
|
||||||
//FIXME
|
{
|
||||||
database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Atomic";
|
/// It's first server run and we need create default and system databases.
|
||||||
|
/// .sql file with database engine will be written for CREATE query.
|
||||||
|
database_attach_query = "CREATE DATABASE " + backQuoteIfNeed(database);
|
||||||
|
}
|
||||||
|
|
||||||
executeCreateQuery(database_attach_query, context, database,
|
executeCreateQuery(database_attach_query, context, database,
|
||||||
database_metadata_file, force_restore_data);
|
database_metadata_file, force_restore_data);
|
||||||
@ -78,7 +78,7 @@ static void loadDatabase(
|
|||||||
#define SYSTEM_DATABASE "system"
|
#define SYSTEM_DATABASE "system"
|
||||||
|
|
||||||
|
|
||||||
void loadMetadata(Context & context)
|
void loadMetadata(Context & context, const String & default_database_name)
|
||||||
{
|
{
|
||||||
String path = context.getPath() + "metadata";
|
String path = context.getPath() + "metadata";
|
||||||
|
|
||||||
@ -108,6 +108,9 @@ void loadMetadata(Context & context)
|
|||||||
databases.emplace(unescapeForFileName(it.name()), it.path().toString());
|
databases.emplace(unescapeForFileName(it.name()), it.path().toString());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (!default_database_name.empty() && !databases.count(default_database_name))
|
||||||
|
databases.emplace(default_database_name, path + "/metadata/" + escapeForFileName(default_database_name));
|
||||||
|
|
||||||
for (const auto & [name, db_path] : databases)
|
for (const auto & [name, db_path] : databases)
|
||||||
loadDatabase(context, name, db_path, has_force_restore_data_flag);
|
loadDatabase(context, name, db_path, has_force_restore_data_flag);
|
||||||
|
|
||||||
|
@ -11,6 +11,6 @@ class Context;
|
|||||||
void loadMetadataSystem(Context & context);
|
void loadMetadataSystem(Context & context);
|
||||||
|
|
||||||
/// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables.
|
/// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables.
|
||||||
void loadMetadata(Context & context);
|
void loadMetadata(Context & context, const String & default_database_name = {});
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
|||||||
&& /s3downloader --dataset-names $DATASETS \
|
&& /s3downloader --dataset-names $DATASETS \
|
||||||
&& chmod 777 -R /var/lib/clickhouse \
|
&& chmod 777 -R /var/lib/clickhouse \
|
||||||
&& clickhouse-client --query "SHOW DATABASES" \
|
&& clickhouse-client --query "SHOW DATABASES" \
|
||||||
&& clickhouse-client --query "CREATE DATABASE datasets ENGINE = Ordinary" \
|
&& clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" \
|
||||||
&& clickhouse-client --query "CREATE DATABASE test" \
|
&& clickhouse-client --query "CREATE DATABASE test" \
|
||||||
&& service clickhouse-server restart && sleep 5 \
|
&& service clickhouse-server restart && sleep 5 \
|
||||||
&& clickhouse-client --query "SHOW TABLES FROM datasets" \
|
&& clickhouse-client --query "SHOW TABLES FROM datasets" \
|
||||||
|
@ -39,7 +39,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
|||||||
service clickhouse-server start && sleep 5 \
|
service clickhouse-server start && sleep 5 \
|
||||||
&& /s3downloader --dataset-names $DATASETS \
|
&& /s3downloader --dataset-names $DATASETS \
|
||||||
&& chmod 777 -R /var/lib/clickhouse \
|
&& chmod 777 -R /var/lib/clickhouse \
|
||||||
&& clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" \
|
&& clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" \
|
||||||
&& clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" \
|
&& clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" \
|
||||||
&& service clickhouse-server restart && sleep 5 \
|
&& service clickhouse-server restart && sleep 5 \
|
||||||
&& clickhouse-client --query "SHOW TABLES FROM datasets" \
|
&& clickhouse-client --query "SHOW TABLES FROM datasets" \
|
||||||
|
Loading…
Reference in New Issue
Block a user