mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 11:32:03 +00:00
Fix tests and bugs
This commit is contained in:
parent
39e79bb1a2
commit
42d0e27bfa
@ -370,12 +370,7 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id)
|
|||||||
|
|
||||||
StorageInMemoryMetadata IStorage::getInMemoryMetadata() const
|
StorageInMemoryMetadata IStorage::getInMemoryMetadata() const
|
||||||
{
|
{
|
||||||
return
|
return StorageInMemoryMetadata(getColumns(), getIndices(), getConstraints());
|
||||||
{
|
|
||||||
.columns = getColumns(),
|
|
||||||
.indices = getIndices(),
|
|
||||||
.constraints = getConstraints(),
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void IStorage::alter(
|
void IStorage::alter(
|
||||||
|
@ -242,11 +242,7 @@ MergeTreeData::MergeTreeData(
|
|||||||
|
|
||||||
StorageInMemoryMetadata MergeTreeData::getInMemoryMetadata() const
|
StorageInMemoryMetadata MergeTreeData::getInMemoryMetadata() const
|
||||||
{
|
{
|
||||||
StorageInMemoryMetadata metadata{
|
StorageInMemoryMetadata metadata(getColumns(), getIndices(), getConstraints());
|
||||||
.columns = getColumns(),
|
|
||||||
.indices = getIndices(),
|
|
||||||
.constraints = getConstraints(),
|
|
||||||
};
|
|
||||||
|
|
||||||
if (partition_by_ast)
|
if (partition_by_ast)
|
||||||
metadata.partition_by_ast = partition_by_ast->clone();
|
metadata.partition_by_ast = partition_by_ast->clone();
|
||||||
|
@ -643,17 +643,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
|||||||
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
|
throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \
|
||||||
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
|
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
|
||||||
|
|
||||||
StorageInMemoryMetadata metadata{
|
StorageInMemoryMetadata metadata(args.columns, indices_description, args.constraints);
|
||||||
.columns = args.columns,
|
metadata.partition_by_ast = partition_by_ast;
|
||||||
.indices = indices_description,
|
metadata.order_by_ast = order_by_ast;
|
||||||
.constraints = args.constraints,
|
metadata.primary_key_ast = primary_key_ast;
|
||||||
.partition_by_ast = partition_by_ast,
|
metadata.ttl_for_table_ast = ttl_table_ast;
|
||||||
.order_by_ast = order_by_ast,
|
metadata.sample_by_ast = sample_by_ast;
|
||||||
.primary_key_ast = primary_key_ast,
|
metadata.settings_ast = settings_ast;
|
||||||
.ttl_for_table_ast = ttl_table_ast,
|
|
||||||
.sample_by_ast = sample_by_ast,
|
|
||||||
.settings_ast = settings_ast,
|
|
||||||
};
|
|
||||||
if (replicated)
|
if (replicated)
|
||||||
return StorageReplicatedMergeTree::create(
|
return StorageReplicatedMergeTree::create(
|
||||||
zookeeper_path, replica_name, args.attach, args.table_id, args.relative_data_path,
|
zookeeper_path, replica_name, args.attach, args.table_id, args.relative_data_path,
|
||||||
|
80
dbms/src/Storages/StorageInMemoryMetadata.cpp
Normal file
80
dbms/src/Storages/StorageInMemoryMetadata.cpp
Normal file
@ -0,0 +1,80 @@
|
|||||||
|
#include <Storages/StorageInMemoryMetadata.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
StorageInMemoryMetadata::StorageInMemoryMetadata(
|
||||||
|
const ColumnsDescription & columns_,
|
||||||
|
const IndicesDescription & indices_,
|
||||||
|
const ConstraintsDescription & constraints_)
|
||||||
|
: columns(columns_)
|
||||||
|
, indices(indices_)
|
||||||
|
, constraints(constraints_)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
StorageInMemoryMetadata::StorageInMemoryMetadata(const StorageInMemoryMetadata & other)
|
||||||
|
: columns(other.columns)
|
||||||
|
, indices(other.indices)
|
||||||
|
, constraints(other.constraints)
|
||||||
|
{
|
||||||
|
if (other.partition_by_ast)
|
||||||
|
partition_by_ast = other.partition_by_ast->clone();
|
||||||
|
if (other.order_by_ast)
|
||||||
|
order_by_ast = other.order_by_ast->clone();
|
||||||
|
if (other.primary_key_ast)
|
||||||
|
primary_key_ast = other.primary_key_ast->clone();
|
||||||
|
if (other.ttl_for_table_ast)
|
||||||
|
ttl_for_table_ast = other.ttl_for_table_ast->clone();
|
||||||
|
if (other.sample_by_ast)
|
||||||
|
sample_by_ast = other.sample_by_ast->clone();
|
||||||
|
if (other.settings_ast)
|
||||||
|
settings_ast = other.settings_ast->clone();
|
||||||
|
if (other.select)
|
||||||
|
select = other.select->clone();
|
||||||
|
}
|
||||||
|
|
||||||
|
StorageInMemoryMetadata & StorageInMemoryMetadata::operator=(const StorageInMemoryMetadata & other)
|
||||||
|
{
|
||||||
|
columns = other.columns;
|
||||||
|
indices = other.indices;
|
||||||
|
constraints = other.constraints;
|
||||||
|
|
||||||
|
|
||||||
|
if (other.partition_by_ast)
|
||||||
|
partition_by_ast = other.partition_by_ast->clone();
|
||||||
|
else
|
||||||
|
partition_by_ast.reset();
|
||||||
|
|
||||||
|
if (other.order_by_ast)
|
||||||
|
order_by_ast = other.order_by_ast->clone();
|
||||||
|
else
|
||||||
|
order_by_ast.reset();
|
||||||
|
|
||||||
|
if (other.primary_key_ast)
|
||||||
|
primary_key_ast = other.primary_key_ast->clone();
|
||||||
|
else
|
||||||
|
primary_key_ast.reset();
|
||||||
|
|
||||||
|
if (other.ttl_for_table_ast)
|
||||||
|
ttl_for_table_ast = other.ttl_for_table_ast->clone();
|
||||||
|
else
|
||||||
|
ttl_for_table_ast.reset();
|
||||||
|
|
||||||
|
if (other.sample_by_ast)
|
||||||
|
sample_by_ast = other.sample_by_ast->clone();
|
||||||
|
else
|
||||||
|
sample_by_ast.reset();
|
||||||
|
|
||||||
|
if (other.settings_ast)
|
||||||
|
settings_ast = other.settings_ast->clone();
|
||||||
|
else
|
||||||
|
settings_ast.reset();
|
||||||
|
|
||||||
|
if (other.select)
|
||||||
|
select = other.select->clone();
|
||||||
|
else
|
||||||
|
select.reset();
|
||||||
|
|
||||||
|
return *this;
|
||||||
|
}
|
||||||
|
}
|
@ -35,6 +35,12 @@ struct StorageInMemoryMetadata
|
|||||||
ASTPtr settings_ast = nullptr;
|
ASTPtr settings_ast = nullptr;
|
||||||
/// SELECT QUERY. Supported for MaterializedView only.
|
/// SELECT QUERY. Supported for MaterializedView only.
|
||||||
ASTPtr select = nullptr;
|
ASTPtr select = nullptr;
|
||||||
|
|
||||||
|
StorageInMemoryMetadata(const StorageInMemoryMetadata & other);
|
||||||
|
StorageInMemoryMetadata() = default;
|
||||||
|
StorageInMemoryMetadata(const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_);
|
||||||
|
|
||||||
|
StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -165,13 +165,9 @@ bool StorageMaterializedView::hasColumn(const String & column_name) const
|
|||||||
|
|
||||||
StorageInMemoryMetadata StorageMaterializedView::getInMemoryMetadata() const
|
StorageInMemoryMetadata StorageMaterializedView::getInMemoryMetadata() const
|
||||||
{
|
{
|
||||||
return
|
StorageInMemoryMetadata result(getColumns(), getIndices(), getConstraints());
|
||||||
{
|
result.select = getSelectQuery();
|
||||||
.columns = getColumns(),
|
return result;
|
||||||
.indices = getIndices(),
|
|
||||||
.constraints = getConstraints(),
|
|
||||||
.select = getSelectQuery(),
|
|
||||||
};
|
|
||||||
}
|
}
|
||||||
|
|
||||||
QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context) const
|
QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context) const
|
||||||
|
@ -411,7 +411,6 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes()
|
|||||||
zookeeper->createIfNotExists(zookeeper_path + "/quorum", String());
|
zookeeper->createIfNotExists(zookeeper_path + "/quorum", String());
|
||||||
zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", String());
|
zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", String());
|
||||||
zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", String());
|
zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", String());
|
||||||
zookeeper->createIfNotExists(zookeeper_path + "/alter_intention_counter", String());
|
|
||||||
|
|
||||||
/// Tracking lag of replicas.
|
/// Tracking lag of replicas.
|
||||||
zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", String());
|
zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", String());
|
||||||
@ -420,10 +419,6 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes()
|
|||||||
/// Mutations
|
/// Mutations
|
||||||
zookeeper->createIfNotExists(zookeeper_path + "/mutations", String());
|
zookeeper->createIfNotExists(zookeeper_path + "/mutations", String());
|
||||||
zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String());
|
zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String());
|
||||||
|
|
||||||
/// ALTERs of the metadata node.
|
|
||||||
zookeeper->createIfNotExists(replica_path + "/metadata", String());
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -485,12 +480,14 @@ void StorageReplicatedMergeTree::checkTableStructure(const String & zookeeper_pr
|
|||||||
old_metadata.checkEquals(metadata_from_zk);
|
old_metadata.checkEquals(metadata_from_zk);
|
||||||
|
|
||||||
Coordination::Stat columns_stat;
|
Coordination::Stat columns_stat;
|
||||||
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &columns_stat));
|
auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_prefix + "/columns", &columns_stat));
|
||||||
|
|
||||||
const ColumnsDescription & old_columns = getColumns();
|
const ColumnsDescription & old_columns = getColumns();
|
||||||
if (columns_from_zk != old_columns)
|
if (columns_from_zk != old_columns)
|
||||||
|
{
|
||||||
throw Exception("Table columns structure in ZooKeeper is different from local table structure", ErrorCodes::INCOMPATIBLE_COLUMNS);
|
throw Exception("Table columns structure in ZooKeeper is different from local table structure", ErrorCodes::INCOMPATIBLE_COLUMNS);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff)
|
void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & metadata_diff)
|
||||||
@ -590,6 +587,7 @@ void StorageReplicatedMergeTree::createReplica()
|
|||||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/parts", "", zkutil::CreateMode::Persistent));
|
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/parts", "", zkutil::CreateMode::Persistent));
|
||||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/flags", "", zkutil::CreateMode::Persistent));
|
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/flags", "", zkutil::CreateMode::Persistent));
|
||||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/is_lost", is_lost_value, zkutil::CreateMode::Persistent));
|
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/is_lost", is_lost_value, zkutil::CreateMode::Persistent));
|
||||||
|
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata", ReplicatedMergeTreeTableMetadata(*this).toString(), zkutil::CreateMode::Persistent));
|
||||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", getColumns().toString(), zkutil::CreateMode::Persistent));
|
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", getColumns().toString(), zkutil::CreateMode::Persistent));
|
||||||
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent));
|
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", std::to_string(metadata_version), zkutil::CreateMode::Persistent));
|
||||||
/// Check version of /replicas to see if there are any replicas created at the same moment of time.
|
/// Check version of /replicas to see if there are any replicas created at the same moment of time.
|
||||||
@ -3255,6 +3253,7 @@ void StorageReplicatedMergeTree::alter(
|
|||||||
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
||||||
params.apply(metadata);
|
params.apply(metadata);
|
||||||
|
|
||||||
|
|
||||||
changeSettings(metadata.settings_ast, table_lock_holder);
|
changeSettings(metadata.settings_ast, table_lock_holder);
|
||||||
|
|
||||||
global_context.getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, metadata);
|
global_context.getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, metadata);
|
||||||
|
@ -1,9 +1,9 @@
|
|||||||
SET send_logs_level = 'none';
|
SET send_logs_level = 'none';
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test.old_style;
|
DROP TABLE IF EXISTS old_style;
|
||||||
CREATE TABLE test.old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192);
|
CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192);
|
||||||
ALTER TABLE test.old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 }
|
ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 }
|
||||||
DROP TABLE test.old_style;
|
DROP TABLE old_style;
|
||||||
|
|
||||||
DROP TABLE IF EXISTS test.summing_r1;
|
DROP TABLE IF EXISTS test.summing_r1;
|
||||||
DROP TABLE IF EXISTS test.summing_r2;
|
DROP TABLE IF EXISTS test.summing_r2;
|
||||||
@ -40,7 +40,7 @@ SELECT '*** Check SHOW CREATE TABLE ***';
|
|||||||
SHOW CREATE TABLE test.summing_r2;
|
SHOW CREATE TABLE test.summing_r2;
|
||||||
|
|
||||||
DETACH TABLE test.summing_r2;
|
DETACH TABLE test.summing_r2;
|
||||||
ALTER TABLE test.summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t); -- { serverError 341 }
|
ALTER TABLE test.summing_r1 ADD COLUMN t UInt32 AFTER z, MODIFY ORDER BY (x, y, t * t) SETTINGS replication_alter_partitions_sync = 2; -- { serverError 341 }
|
||||||
ATTACH TABLE test.summing_r2;
|
ATTACH TABLE test.summing_r2;
|
||||||
SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***';
|
SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***';
|
||||||
SHOW CREATE TABLE test.summing_r2;
|
SHOW CREATE TABLE test.summing_r2;
|
||||||
|
Loading…
Reference in New Issue
Block a user