minor improvements

This commit is contained in:
Alexander Tokmakov 2019-12-27 22:30:22 +03:00
parent 327cc4655f
commit 94931f059d
20 changed files with 35 additions and 67 deletions

View File

@ -793,7 +793,7 @@ Dependencies Context::getDependencies(const StorageID & from) const
checkDatabaseAccessRightsImpl(db);
}
ViewDependencies::const_iterator iter = shared->view_dependencies.find(StorageID(db, from.table_name));
ViewDependencies::const_iterator iter = shared->view_dependencies.find(StorageID(db, from.table_name, from.uuid));
if (iter == shared->view_dependencies.end())
return {};

View File

@ -2,6 +2,7 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Core/UUID.h>
namespace DB
@ -15,7 +16,7 @@ class ASTQueryWithTableAndOutput : public ASTQueryWithOutput
public:
String database;
String table;
String uuid;
UUID uuid;
bool temporary{false};
protected:

View File

@ -462,24 +462,15 @@ DB::CompressionMethod IStorage::chooseCompressionMethod(const String & uri, cons
throw Exception("Only auto, none, gzip supported as compression method", ErrorCodes::NOT_IMPLEMENTED);
}
StorageID IStorage::getStorageID(std::unique_lock<std::mutex> * id_lock) const
StorageID IStorage::getStorageID() const
{
std::unique_lock<std::mutex> lock;
if (!id_lock)
lock = std::unique_lock(id_mutex);
else if (!*id_lock)
*id_lock = std::unique_lock(id_mutex);
return storage_id;
}
void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name,
std::unique_lock<std::mutex> * id_lock)
void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name)
{
std::unique_lock<std::mutex> lock;
if (!id_lock)
lock = std::unique_lock(id_mutex);
else if (!*id_lock)
*id_lock = std::unique_lock(id_mutex);
storage_id.database_name = new_database_name;
storage_id.table_name = new_table_name;
}

View File

@ -88,7 +88,7 @@ public:
virtual std::string getName() const = 0;
/// The name of the table.
StorageID getStorageID(std::unique_lock<std::mutex> * lock = nullptr) const;
StorageID getStorageID() const;
/// Returns true if the storage receives data from a remote server or servers.
virtual bool isRemote() const { return false; }
@ -317,7 +317,7 @@ public:
* Just updates names of database and table without moving any data on disk
* Can be called directly only from DatabaseAtomic.
*/
virtual void renameInMemory(const String & new_database_name, const String & new_table_name, std::unique_lock<std::mutex> * id_lock = nullptr);
virtual void renameInMemory(const String & new_database_name, const String & new_table_name);
/** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters.
* This method must fully execute the ALTER query, taking care of the locks itself.

View File

@ -5,7 +5,7 @@
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#include <IO/WriteHelpers.h>
#include <Storages/IStorage.h>
#include <Storages/StorageID.h>
namespace DB
{

View File

@ -1,6 +1,8 @@
#pragma once
#include <Core/Types.h>
#include <Core/UUID.h>
#include <Common/quoteString.h>
#include <IO/WriteHelpers.h>
#include <tuple>
namespace DB
@ -17,16 +19,12 @@ struct StorageID
{
String database_name;
String table_name;
String uuid;
UUID uuid;
//StorageID() = delete;
StorageID() = default;
//TODO StorageID(const ASTPtr & query_with_one_table, const Context & context) to get db and table names (and maybe uuid) from query
//But there are a lot of different ASTs with db and table name
//And it looks like it depends on https://github.com/ClickHouse/ClickHouse/pull/7774
StorageID(const String & database, const String & table, const String & uuid_ = {})
StorageID(const String & database, const String & table, UUID uuid_ = UUID{UInt128(0, 0)})
: database_name(database), table_name(table), uuid(uuid_)
{
}
@ -40,45 +38,42 @@ struct StorageID
String getNameForLogs() const
{
assert_valid();
return (database_name.empty() ? "" : backQuoteIfNeed(database_name) + ".") + backQuoteIfNeed(table_name) + (uuid.empty() ? "" : " (UUID " + uuid + ")");
}
String getId() const
{
//if (uuid.empty())
return getFullTableName();
//else
// return uuid;
return (database_name.empty() ? "" : backQuoteIfNeed(database_name) + ".") + backQuoteIfNeed(table_name) + (hasUUID() ? "" : " (UUID " + toString(uuid) + ")");
}
bool operator<(const StorageID & rhs) const
{
assert_valid();
/// It's needed for ViewDependencies
if (uuid.empty() && rhs.uuid.empty())
if (!hasUUID() && !rhs.hasUUID())
/// If both IDs don't have UUID, compare them like pair of strings
return std::tie(database_name, table_name) < std::tie(rhs.database_name, rhs.table_name);
else if (!uuid.empty() && !rhs.uuid.empty())
else if (hasUUID() && rhs.hasUUID())
/// If both IDs have UUID, compare UUIDs and ignore database and table name
return uuid < rhs.uuid;
else
/// All IDs without UUID are less, then all IDs with UUID
return uuid.empty();
return !hasUUID();
}
bool empty() const
{
return table_name.empty() || (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && uuid.empty());
return table_name.empty() || (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && !hasUUID());
}
void assert_valid() const
{
if (empty())
throw Exception("empty table name", ErrorCodes::LOGICAL_ERROR);
if (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && uuid.empty() && !database_name.empty())
if (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && !hasUUID() && !database_name.empty())
throw Exception("unexpected database name", ErrorCodes::LOGICAL_ERROR);
}
bool hasUUID() const
{
return uuid != UUID{UInt128(0, 0)};
}
};
}

View File

@ -109,8 +109,6 @@ private:
/// You can not call with a write locked `rwlock`.
void loadMarks();
/// The order of adding files should not change: it corresponds to the order of the columns in the marks file.
/** For normal columns, the number of rows in the block is specified in the marks.
* For array columns and nested structures, there are more than one group of marks that correspond to different files
* - for elements (file name.bin) - the total number of array elements in the block is specified,

View File

@ -7,19 +7,16 @@
#include <DataStreams/ConcatBlockInputStream.h>
#include <DataStreams/materializeBlock.h>
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/FilterBlockInputStream.h>
#include <Storages/StorageMerge.h>
#include <Storages/StorageFactory.h>
#include <Storages/VirtualColumnUtils.h>
#include <Storages/AlterCommands.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExpressionList.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Common/typeid_cast.h>
@ -27,7 +24,6 @@
#include <Databases/IDatabase.h>
#include <ext/range.h>
#include <algorithm>
#include <Parsers/ASTFunction.h>
#include <Parsers/queryToString.h>
@ -37,7 +33,6 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_PREWHERE;
extern const int INCOMPATIBLE_SOURCE_TABLES;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE;
@ -281,10 +276,8 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
SelectQueryInfo modified_query_info = query_info;
modified_query_info.query = query_info.query->clone();
StorageID table_id;
if (storage)
table_id = storage->getStorageID();
VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_id.table_name);
String table_name = storage ? storage->getStorageID().table_name : "";
VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name);
if (!storage)
return BlockInputStreams{
@ -304,7 +297,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
}
else if (processed_stage > storage->getQueryProcessingStage(modified_context))
{
modified_query_info.query->as<ASTSelectQuery>()->replaceDatabaseAndTable(source_database, table_id.table_name);
modified_query_info.query->as<ASTSelectQuery>()->replaceDatabaseAndTable(source_database, table_name);
/// Maximum permissible parallelism is streams_num
modified_context.getSettingsRef().max_threads = UInt64(streams_num);
@ -335,7 +328,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
{
if (has_table_virtual_column)
source_stream = std::make_shared<AddingConstColumnBlockInputStream<String>>(
source_stream, std::make_shared<DataTypeString>(), table_id.table_name, "_table");
source_stream, std::make_shared<DataTypeString>(), table_name, "_table");
/// Subordinary tables could have different but convertible types, like numeric types of different width.
/// We must return streams with structure equals to structure of Merge table.

View File

@ -660,7 +660,6 @@ bool StorageMergeTree::merge(
}
auto table_id = getStorageID();
//FIXME
MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_part);
/// Logging

View File

@ -574,7 +574,6 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
}
auto table_id = getStorageID();
//FIXME
global_context.getDatabase(table_id.database_name)->alterTable(global_context, table_id.table_name, new_columns, new_indices, new_constraints, storage_modifier);
/// Even if the primary/sorting keys didn't change we must reinitialize it

View File

@ -143,7 +143,7 @@ StorageS3::StorageS3(
const ConstraintsDescription & constraints_,
Context & context_,
const String & compression_method_ = "")
: IStorage({table_id_}, columns_)
: IStorage(table_id_, columns_)
, uri(uri_)
, context_global(context_)
, format_name(format_name_)

View File

@ -292,9 +292,6 @@ CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Conte
void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
{
if (getStorageID().table_name.empty()) //FIXME how can it be empty?
throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR);
std::shared_lock<std::shared_mutex> lock(rwlock);
auto file = Poco::File(path);

View File

@ -422,9 +422,6 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context
void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
{
if (getStorageID().table_name.empty()) //FIXME how can it be empty?
throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR);
std::unique_lock<std::shared_mutex> lock(rwlock);
auto file = Poco::File(path);

View File

@ -51,7 +51,6 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context &
auto table_id = table_data->getStorageID();
if (!graphite_configs.count(config_name))
{
//TODO add uuid
Config new_config =
{
table_data->merging_params.graphite_params,

View File

@ -98,7 +98,6 @@ public:
protected:
Block readImpl() override
{
//FIXME add uuids
if (done)
return {};

View File

@ -16,7 +16,7 @@ def started_cluster():
def test_file_path_escaping(started_cluster):
node.query('CREATE DATABASE IF NOT EXISTS test ENGINE = Ordinary')
node.query('CREATE DATABASE IF NOT EXISTS test')
node.query('''
CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32)
ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id`;

View File

@ -14,7 +14,7 @@ path_to_data = '/var/lib/clickhouse/'
def started_cluster():
try:
cluster.start()
q('CREATE DATABASE test ENGINE = Ordinary')
q('CREATE DATABASE test')
yield cluster

View File

@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1);
SELECT * FROM deduplication;
DETACH TABLE deduplication;
ATTACH TABLE deduplication;-- (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1);
ATTACH TABLE deduplication (d Date DEFAULT '2015-01-01', x Int8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/deduplication', 'r1', d, x, 1);
SELECT * FROM deduplication;

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS table_old;
DROP TABLE IF EXISTS table_new;
CREATE TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0;
CREATE TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=0;
CREATE TABLE table_new (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'new') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1;
SET insert_quorum=2;
@ -17,8 +17,8 @@ SELECT * FROM table_new ORDER BY k;
SELECT 'DETACH';
DETACH TABLE table_old;
ATTACH TABLE table_old UUID '00000611-abcd-4000-8000-123456789ab6c' (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1;
ATTACH TABLE table_old (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'old') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1;
SELECT * FROM table_old ORDER BY k;
DROP TABLE IF EXISTS table_old;
DROP TABLE IF EXISTS table_new;
DROP TABLE IF EXISTS table_new;

View File

@ -24,7 +24,7 @@ GROUP BY loyalty
ORDER BY loyalty ASC;
DETACH TABLE join;
ATTACH TABLE join;-- (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID);
ATTACH TABLE join (UserID UInt64, loyalty Int8) ENGINE = Join(ANY, INNER, UserID);
SELECT
loyalty,