This commit is contained in:
Alexander Tokmakov 2019-12-12 15:30:31 +03:00
parent 42296cc80a
commit 3b710feeb6
9 changed files with 18 additions and 18 deletions

View File

@ -29,7 +29,7 @@ namespace ErrorCodes
extern const int TABLE_IS_DROPPED; extern const int TABLE_IS_DROPPED;
} }
IStorage::IStorage(StorageID id_, ColumnsDescription virtuals_) : id(std::move(id_)), virtuals(std::move(virtuals_)) IStorage::IStorage(StorageID storage_id_, ColumnsDescription virtuals_) : storage_id(std::move(storage_id_)), virtuals(std::move(virtuals_))
{ {
} }
@ -469,7 +469,7 @@ StorageID IStorage::getStorageID(std::unique_lock<std::mutex> * id_lock) const
lock = std::unique_lock(id_mutex); lock = std::unique_lock(id_mutex);
else if (!*id_lock) else if (!*id_lock)
*id_lock = std::unique_lock(id_mutex); *id_lock = std::unique_lock(id_mutex);
return id; return storage_id;
} }
void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name, void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name,
@ -480,8 +480,8 @@ void IStorage::renameInMemory(const String & new_database_name, const String & n
lock = std::unique_lock(id_mutex); lock = std::unique_lock(id_mutex);
else if (!*id_lock) else if (!*id_lock)
*id_lock = std::unique_lock(id_mutex); *id_lock = std::unique_lock(id_mutex);
id.database_name = new_database_name; storage_id.database_name = new_database_name;
id.table_name = new_table_name; storage_id.table_name = new_table_name;
} }
} }

View File

@ -77,7 +77,7 @@ class IStorage : public std::enable_shared_from_this<IStorage>, public TypePromo
{ {
public: public:
IStorage() = delete; IStorage() = delete;
explicit IStorage(StorageID id_) : id(std::move(id_)) {} explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {}
IStorage(StorageID id_, ColumnsDescription virtuals_); IStorage(StorageID id_, ColumnsDescription virtuals_);
virtual ~IStorage() = default; virtual ~IStorage() = default;
@ -167,7 +167,7 @@ protected: /// still thread-unsafe part.
IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const; IDatabase::ASTModifier getSettingsModifier(const SettingsChanges & new_changes) const;
private: private:
StorageID id; StorageID storage_id;
mutable std::mutex id_mutex; mutable std::mutex id_mutex;
ColumnsDescription columns; /// combined real and virtual columns ColumnsDescription columns; /// combined real and virtual columns
const ColumnsDescription virtuals = {}; const ColumnsDescription virtuals = {};

View File

@ -345,18 +345,18 @@ BlockInputStreams StorageDistributed::read(
if (has_sharding_key) if (has_sharding_key)
{ {
auto smaller_cluster = skipUnusedShards(cluster, query_info); auto smaller_cluster = skipUnusedShards(cluster, query_info);
auto storage_id = getStorageID(); auto table_id = getStorageID();
if (smaller_cluster) if (smaller_cluster)
{ {
cluster = smaller_cluster; cluster = smaller_cluster;
LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": "
"Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): " "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): "
" " << makeFormattedListOfShards(cluster)); " " << makeFormattedListOfShards(cluster));
} }
else else
{ {
LOG_DEBUG(log, "Reading from " << storage_id.getNameForLogs() << ": " LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << ": "
"Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster"); "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster");
} }
} }

View File

@ -56,14 +56,14 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context &
{ {
table_data->merging_params.graphite_params, table_data->merging_params.graphite_params,
{ table_id.database_name }, { table_id.database_name },
{ table_id.database_name }, { table_id.table_name },
}; };
graphite_configs.emplace(config_name, new_config); graphite_configs.emplace(config_name, new_config);
} }
else else
{ {
graphite_configs[config_name].databases.emplace_back(table_id.database_name); graphite_configs[config_name].databases.emplace_back(table_id.database_name);
graphite_configs[config_name].tables.emplace_back(table_id.database_name); graphite_configs[config_name].tables.emplace_back(table_id.table_name);
} }
} }
} }

View File

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

View File

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

View File

@ -21,7 +21,7 @@ INSERT INTO deduplication (x) VALUES (1);
SELECT * FROM deduplication; SELECT * FROM deduplication;
DETACH TABLE 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; SELECT * FROM deduplication;

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS table_old; DROP TABLE IF EXISTS table_old;
DROP TABLE IF EXISTS table_new; DROP TABLE IF EXISTS table_new;
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_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_new (k UInt64, d Array(String)) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/checksums_test', 'new') ORDER BY k SETTINGS use_minimalistic_checksums_in_zookeeper=1; 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; SET insert_quorum=2;
@ -17,8 +17,8 @@ SELECT * FROM table_new ORDER BY k;
SELECT 'DETACH'; SELECT 'DETACH';
DETACH TABLE table_old; DETACH TABLE table_old;
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; 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;
SELECT * FROM table_old ORDER BY k; SELECT * FROM table_old ORDER BY k;
DROP TABLE IF EXISTS table_old; 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; ORDER BY loyalty ASC;
DETACH TABLE join; 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 SELECT
loyalty, loyalty,