This commit is contained in:
Alexander Tokmakov 2020-03-18 02:51:35 +03:00
parent 960ba63104
commit 5438cef23a
17 changed files with 39 additions and 69 deletions

View File

@ -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 (...)

View File

@ -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)

View File

@ -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"; }

View File

@ -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;

View File

@ -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);

View File

@ -15,7 +15,6 @@ namespace DB
class Context;
/// A base class for databases that manage their own list of tables.
class DatabaseWithOwnTablesBase : public IDatabase
{

View File

@ -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());
}
pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back());
if (*pos == ':')
state = Format;
@ -107,16 +100,7 @@ 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());
}
}
column_idx = idx_by_name(column_names.back());
}
format_idx_to_column_idx.emplace_back(column_idx);
break;

View File

@ -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;

View File

@ -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).

View File

@ -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();
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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 : "")

View File

@ -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);

View File

@ -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);
}

View File

@ -1,4 +1,4 @@
CREATE DATABASE test_00751;
CREATE DATABASE IF NOT EXISTS test_00751;
USE test_00751;
DROP TABLE IF EXISTS t_00751;

View File

@ -49,7 +49,7 @@ ORDER BY StartDate;
SELECT
StartDate,
sum(Sign) AS Visits,
sum(Sign) AS Visits,
uniq(UserID) AS Users
FROM test.visits
WHERE CounterID = 942285