This commit is contained in:
Alexander Tokmakov 2020-03-18 20:38:52 +03:00
parent 5438cef23a
commit 3d058c2060
12 changed files with 39 additions and 26 deletions

View File

@ -139,10 +139,10 @@ void DatabaseAtomic::dropTable(const Context & context, const String & table_nam
{
LOG_INFO(log, "Mark table " + table->getStorageID().getNameForLogs() + " to drop.");
/// Context:getPath acquires lock
auto data_path = context.getPath() + table_data_path_relative;
auto table_data_path = context.getPath() + table_data_path_relative;
std::lock_guard lock(tables_to_drop_mutex);
time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
tables_to_drop.push_back({table, data_path, current_time});
tables_to_drop.push_back({table, table_data_path, current_time});
}
}
catch (...)

View File

@ -246,7 +246,9 @@ void DatabaseOnDisk::renameTable(
TableStructureWriteLockHolder table_lock;
String table_metadata_path;
ASTPtr attach_query;
StoragePtr table = detachTable(table_name);
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
StoragePtr table = tryGetTable(context, table_name);
detachTable(table_name);
try
{
table_lock = table->lockExclusively(context.getCurrentQueryId());
@ -451,6 +453,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
if (Poco::Path(table_name).makeFile().getExtension() == "sql")
table_name = Poco::Path(table_name).makeFile().getBaseName();
table_name = unescapeForFileName(table_name);
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER)
LOG_WARNING(log, "File " << metadata_file_path << " contains both UUID and table name. "

View File

@ -603,16 +603,11 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create,
if (!create.attach && create.uuid == UUIDHelpers::Nil && database->getEngineName() == "Atomic")
create.uuid = UUIDHelpers::generateV4();
data_path = database->getTableDataPath(create);
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
* If table doesnt exist, one thread is creating table, while others wait in DDLGuard.
*/
guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name);
if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists())
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
if (database->isTableExist(context, table_name))
{
@ -633,6 +628,10 @@ bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create,
else
throw Exception("Table " + create.database + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
data_path = database->getTableDataPath(create);
if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists())
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
}
else
{

View File

@ -34,9 +34,11 @@ namespace ErrorCodes
extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW;
}
static inline String generateInnerTableName(const String & table_name)
static inline String generateInnerTableName(const StorageID & view_id)
{
return ".inner." + table_name;
if (view_id.hasUUID())
return ".inner." + toString(view_id.uuid);
return ".inner." + view_id.getTableName();
}
static StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, const Context & context, bool add_default_db = true)
@ -128,14 +130,14 @@ StorageMaterializedView::StorageMaterializedView(
else if (attach_)
{
/// If there is an ATTACH request, then the internal table must already be created.
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID().table_name));
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID()));
}
else
{
/// We will create a query to create an internal table.
auto manual_create_query = std::make_shared<ASTCreateQuery>();
manual_create_query->database = getStorageID().database_name;
manual_create_query->table = generateInnerTableName(getStorageID().table_name);
manual_create_query->table = generateInnerTableName(getStorageID());
auto new_columns_list = std::make_shared<ASTColumns>();
new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr());
@ -327,9 +329,10 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co
void StorageMaterializedView::renameInMemory(const String & new_database_name, const String & new_table_name)
{
if (has_inner_table && tryGetTargetTable())
auto old_table_id = getStorageID();
if (has_inner_table && tryGetTargetTable() && !old_table_id.hasUUID())
{
auto new_target_table_name = generateInnerTableName(new_table_name);
auto new_target_table_name = generateInnerTableName({new_database_name, new_table_name});
auto rename = std::make_shared<ASTRenameQuery>();
ASTRenameQuery::Table from;
@ -349,7 +352,6 @@ void StorageMaterializedView::renameInMemory(const String & new_database_name, c
target_table_id.table_name = new_target_table_name;
}
auto old_table_id = getStorageID();
IStorage::renameInMemory(new_database_name, new_table_name);
// TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated
DatabaseCatalog::instance().updateDependency(select_table_id, old_table_id, select_table_id, getStorageID());

View File

@ -61,18 +61,18 @@ def test_two_files(started_cluster):
def test_single_file_old(started_cluster):
node.query("create table distr_3 (x UInt64, s String) engine = Distributed('test_cluster', database, table)")
node.query("insert into distr_3 values (1, 'a'), (2, 'bb'), (3, 'ccc')")
node.query("create table test.distr_3 (x UInt64, s String) engine = Distributed('test_cluster', database, table)")
node.query("insert into test.distr_3 values (1, 'a'), (2, 'bb'), (3, 'ccc')")
query = "select * from file('/var/lib/clickhouse/data/default/distr_3/default@not_existing:9000/1.bin', 'Distributed')"
query = "select * from file('/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin', 'Distributed')"
out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query])
assert out == '1\ta\n2\tbb\n3\tccc\n'
query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/default/distr_3/default@not_existing:9000/1.bin');" \
query = "create table t (dummy UInt32) engine = File('Distributed', '/var/lib/clickhouse/data/test/distr_3/default@not_existing:9000/1.bin');" \
"select * from t"
out = node.exec_in_container(['/usr/bin/clickhouse', 'local', '--stacktrace', '-q', query])
assert out == '1\ta\n2\tbb\n3\tccc\n'
node.query("drop table distr_3")
node.query("drop table test.distr_3")

View File

@ -2,6 +2,7 @@
<yandex>
<profiles>
<default>
<default_database_engine>Ordinary</default_database_engine>
</default>
</profiles>

View File

@ -2,4 +2,4 @@
<merge_tree>
<min_rows_for_wide_part>512</min_rows_for_wide_part>
</merge_tree>
</yandex>
</yandex>

View File

@ -2,6 +2,7 @@
<profiles>
<default>
<optimize_trivial_count_query>0</optimize_trivial_count_query>
<default_database_engine>Ordinary</default_database_engine>
</default>
</profiles>
</yandex>
</yandex>

View File

@ -0,0 +1,7 @@
<yandex>
<profiles>
<default>
<optimize_trivial_count_query>0</optimize_trivial_count_query>
</default>
</profiles>
</yandex>

View File

@ -168,7 +168,7 @@ def test_different_part_types_on_replicas(start_cluster, table, part_type):
"WHERE table = '{}' AND active GROUP BY part_type ORDER BY part_type".format(table))) == TSV(expected)
node7 = cluster.add_instance('node7', config_dir="configs", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True)
node7 = cluster.add_instance('node7', config_dir="configs_old", with_zookeeper=True, image='yandex/clickhouse-server:19.17.8.54', stay_alive=True, with_installed_binary=True)
node8 = cluster.add_instance('node8', config_dir="configs", with_zookeeper=True)
settings7 = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760}

View File

@ -24,7 +24,7 @@ def started_cluster():
cluster.start()
instance.query('''
CREATE DATABASE mydb;
CREATE DATABASE mydb ENGINE=Ordinary;
CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1);
@ -42,7 +42,7 @@ def started_cluster():
INSERT INTO mydb.`.filtered_table4` values (0, 0), (0, 1), (1, 0), (1, 1);
''')
instance2.query('''
CREATE DATABASE mydb;
CREATE DATABASE mydb ENGINE=Ordinary;
CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a;
INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1);

View File

@ -1,5 +1,5 @@
DROP DATABASE IF EXISTS some_tests;
CREATE DATABASE some_tests;
CREATE DATABASE some_tests ENGINE=Ordinary;
create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str;
show tables from some_tests;