mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
fixes
This commit is contained in:
parent
960ba63104
commit
5438cef23a
@ -558,7 +558,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)
|
||||
attachSystemTablesServer(*DatabaseCatalog::instance().getSystemDatabase(), has_zookeeper);
|
||||
/// Then, load remaining databases
|
||||
loadMetadata(*global_context);
|
||||
loadMetadata(*global_context, default_database);
|
||||
DatabaseCatalog::instance().loadDatabases();
|
||||
}
|
||||
catch (...)
|
||||
|
@ -84,18 +84,21 @@ void DatabaseAtomic::drop(const Context &)
|
||||
", which is used by " + std::to_string(it->table.use_count() - 1) + " queries. "
|
||||
"Client should retry later.", ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
|
||||
for (auto & table : tables_to_drop)
|
||||
{
|
||||
try
|
||||
{
|
||||
dropTableFinally(table);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Cannot drop table. Metadata " + table.data_path + " will be removed forcefully. "
|
||||
"Garbage may be left in /store directory and ZooKeeper.");
|
||||
}
|
||||
}
|
||||
//FIXME maybe make `tables_to_drop` global for all Atomic databases?
|
||||
|
||||
//for (auto & table : tables_to_drop)
|
||||
//{
|
||||
// try
|
||||
// {
|
||||
// /// IStorage::drop() may use DatabaseCatalog, so databases mutex will be acquired (possible deadlock here)
|
||||
// dropTableFinally(table);
|
||||
// }
|
||||
// catch (...)
|
||||
// {
|
||||
// tryLogCurrentException(log, "Cannot drop table. Metadata " + table.data_path + " will be removed forcefully. "
|
||||
// "Garbage may be left in /store directory and ZooKeeper.");
|
||||
// }
|
||||
//}
|
||||
Poco::File(getMetadataPath()).remove(true);
|
||||
}
|
||||
|
||||
@ -172,9 +175,10 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
||||
table->renameInMemory(to_database.getDatabaseName(), to_table_name);
|
||||
|
||||
/// NOTE Non-atomic.
|
||||
to_database.attachTable(to_table_name, table, getTableDataPath(table_name));
|
||||
auto path = getTableDataPath(table_name);
|
||||
detachTable(table_name);
|
||||
Poco::File(getObjectMetadataPath(table_name)).renameTo(to_database.getObjectMetadataPath(to_table_name));
|
||||
to_database.attachTable(to_table_name, table, path);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag)
|
||||
|
@ -19,8 +19,7 @@ namespace DB
|
||||
class DatabaseMemory : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
public:
|
||||
//FIXME default name
|
||||
DatabaseMemory(const String & name_ = "_temporary_and_external_tables");
|
||||
DatabaseMemory(const String & name_);
|
||||
|
||||
String getEngineName() const override { return "Memory"; }
|
||||
|
||||
|
@ -22,7 +22,6 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
|
||||
void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, const Context & context)
|
||||
{
|
||||
String full_name = getDatabaseName() + "." + dictionary_name;
|
||||
@ -60,7 +59,6 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
|
||||
|
||||
/** The code is based on the assumption that all threads share the same order of operations:
|
||||
* - create the .sql.tmp file;
|
||||
* - add the dictionary to ExternalDictionariesLoader;
|
||||
|
@ -83,8 +83,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
|
||||
auto table_id = res->getStorageID();
|
||||
if (table_id.hasUUID())
|
||||
{
|
||||
/// For now it's the only database, which contains storages with UUID
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE);
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
|
||||
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
|
||||
}
|
||||
|
||||
@ -104,8 +103,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
|
||||
auto table_id = table->getStorageID();
|
||||
if (table_id.hasUUID())
|
||||
{
|
||||
/// For now it's the only database, which contains storages with UUID
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE);
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
|
||||
DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table);
|
||||
}
|
||||
}
|
||||
@ -123,7 +121,13 @@ void DatabaseWithOwnTablesBase::shutdown()
|
||||
|
||||
for (const auto & kv : tables_snapshot)
|
||||
{
|
||||
auto table_id = kv.second->getStorageID();
|
||||
kv.second->shutdown();
|
||||
if (table_id.hasUUID())
|
||||
{
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
|
||||
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
|
||||
}
|
||||
}
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
|
@ -15,7 +15,6 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
|
||||
|
||||
/// A base class for databases that manage their own list of tables.
|
||||
class DatabaseWithOwnTablesBase : public IDatabase
|
||||
{
|
||||
|
@ -77,14 +77,7 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum
|
||||
|
||||
case Column:
|
||||
column_names.emplace_back();
|
||||
try
|
||||
{
|
||||
pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back());
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
throwInvalidFormat(e.message(), columnsCount());
|
||||
}
|
||||
|
||||
if (*pos == ':')
|
||||
state = Format;
|
||||
@ -107,17 +100,8 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum
|
||||
errno = 0;
|
||||
column_idx = strtoull(column_names.back().c_str(), &col_idx_end, 10);
|
||||
if (col_idx_end != column_names.back().c_str() + column_names.back().size() || errno)
|
||||
{
|
||||
try
|
||||
{
|
||||
column_idx = idx_by_name(column_names.back());
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
throwInvalidFormat(e.message(), columnsCount());
|
||||
}
|
||||
}
|
||||
}
|
||||
format_idx_to_column_idx.emplace_back(column_idx);
|
||||
break;
|
||||
|
||||
|
@ -58,8 +58,6 @@
|
||||
#include <Common/RemoteHostFilter.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event ContextLock;
|
||||
@ -930,7 +928,6 @@ StoragePtr Context::getViewSource()
|
||||
return view_source;
|
||||
}
|
||||
|
||||
|
||||
Settings Context::getSettings() const
|
||||
{
|
||||
return settings;
|
||||
|
@ -88,7 +88,6 @@ class ShellCommand;
|
||||
class ICompressionCodec;
|
||||
class AccessControlManager;
|
||||
class SettingsConstraints;
|
||||
struct StorageID;
|
||||
class RemoteHostFilter;
|
||||
struct StorageID;
|
||||
class IDisk;
|
||||
@ -131,8 +130,6 @@ struct IHostContext
|
||||
|
||||
using IHostContextPtr = std::shared_ptr<IHostContext>;
|
||||
|
||||
struct TemporaryTableHolder;
|
||||
|
||||
/** A set of known objects that can be used in the query.
|
||||
* Consists of a shared part (always common to all sessions and queries)
|
||||
* and copied part (which can be its own for each session or query).
|
||||
|
@ -126,11 +126,7 @@ void DatabaseCatalog::shutdown()
|
||||
|
||||
|
||||
std::lock_guard lock(databases_mutex);
|
||||
for (auto & elem : uuid_map)
|
||||
{
|
||||
std::lock_guard map_lock(elem.mutex);
|
||||
elem.map.clear();
|
||||
}
|
||||
assert(std::find_if_not(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { return elem.map.empty(); }) == uuid_map.end());
|
||||
databases.clear();
|
||||
view_dependencies.clear();
|
||||
}
|
||||
|
@ -105,12 +105,6 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
engine->name = old_style_database ? "Ordinary" : "Atomic";
|
||||
storage->set(storage->engine, engine);
|
||||
create.set(create.storage, storage);
|
||||
|
||||
if (database_name == "datasets")
|
||||
{
|
||||
//FIXME it's just to run stateful and stress tests without updating docker images
|
||||
engine->name = "Ordinary";
|
||||
}
|
||||
}
|
||||
else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty()))
|
||||
{
|
||||
@ -559,7 +553,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
|
||||
if (!create.temporary && create.database.empty())
|
||||
create.database = current_database;
|
||||
if (!create.to_table_id.empty() && create.to_table_id.database_name.empty())
|
||||
if (create.to_table_id && create.to_table_id.database_name.empty())
|
||||
create.to_table_id.database_name = current_database;
|
||||
|
||||
if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view))
|
||||
@ -587,7 +581,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create,
|
||||
String data_path;
|
||||
DatabasePtr database;
|
||||
|
||||
const String & table_name = create.table;
|
||||
const String table_name = create.table;
|
||||
bool need_add_to_database = !create.temporary;
|
||||
if (need_add_to_database)
|
||||
{
|
||||
@ -642,7 +636,7 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create,
|
||||
}
|
||||
else
|
||||
{
|
||||
if (context.tryResolveStorageID({"", table_name}, Context::ResolveExternal) && create.if_not_exists)
|
||||
if (create.if_not_exists && context.tryResolveStorageID({"", table_name}, Context::ResolveExternal))
|
||||
return false;
|
||||
|
||||
auto temporary_table = TemporaryTableHolder(context, properties.columns, query_ptr);
|
||||
@ -803,7 +797,7 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const
|
||||
}
|
||||
|
||||
if (create.to_table_id)
|
||||
required_access.emplace_back(AccessType::INSERT, create.to_table_id.getDatabaseName(), create.to_table_id.getTableName());
|
||||
required_access.emplace_back(AccessType::INSERT, create.to_table_id.database_name, create.to_table_id.table_name);
|
||||
|
||||
return required_access;
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ static void loadDatabase(
|
||||
|
||||
if (Poco::File(database_metadata_file).exists())
|
||||
{
|
||||
/// There are .sql file with database creation statement.
|
||||
/// There is .sql file with database creation statement.
|
||||
ReadBufferFromFile in(database_metadata_file, 1024);
|
||||
readStringUntilEOF(database_attach_query, in);
|
||||
}
|
||||
|
@ -258,7 +258,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
|
||||
as_table_function->formatImpl(settings, state, frame);
|
||||
}
|
||||
if (!to_table_id.empty())
|
||||
if (to_table_id)
|
||||
{
|
||||
settings.ostr
|
||||
<< (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "")
|
||||
|
@ -82,7 +82,6 @@ public:
|
||||
/// Removes temporary data in local filesystem.
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void renameOnDisk(const String & new_path_to_table_data);
|
||||
|
||||
|
@ -10,8 +10,7 @@ namespace DB
|
||||
StoragePtr TableFunctionFile::getStorage(
|
||||
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const
|
||||
{
|
||||
StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method,
|
||||
columns, ConstraintsDescription{}, global_context};
|
||||
StorageFile::CommonArguments args{StorageID(getDatabaseName(), table_name), format, compression_method, columns, ConstraintsDescription{}, global_context};
|
||||
|
||||
return StorageFile::create(source, global_context.getUserFilesPath(), args);
|
||||
}
|
||||
|
@ -1,4 +1,4 @@
|
||||
CREATE DATABASE test_00751;
|
||||
CREATE DATABASE IF NOT EXISTS test_00751;
|
||||
USE test_00751;
|
||||
|
||||
DROP TABLE IF EXISTS t_00751;
|
||||
|
Loading…
Reference in New Issue
Block a user