mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
minor improvements
This commit is contained in:
parent
327cc4655f
commit
94931f059d
@ -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 {};
|
||||
|
||||
|
@ -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:
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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)};
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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_)
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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,
|
||||
|
@ -98,7 +98,6 @@ public:
|
||||
protected:
|
||||
Block readImpl() override
|
||||
{
|
||||
//FIXME add uuids
|
||||
if (done)
|
||||
return {};
|
||||
|
||||
|
@ -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`;
|
||||
|
@ -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
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
@ -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,
|
||||
|
Loading…
Reference in New Issue
Block a user