Fix tests and bugs

This commit is contained in:
alesapin 2020-02-14 16:17:50 +03:00
parent 39e79bb1a2
commit 42d0e27bfa
9 changed files with 110 additions and 41 deletions

View File

@ -370,12 +370,7 @@ TableStructureWriteLockHolder IStorage::lockExclusively(const String & query_id)
StorageInMemoryMetadata IStorage::getInMemoryMetadata() const
{
return
{
.columns = getColumns(),
.indices = getIndices(),
.constraints = getConstraints(),
};
return StorageInMemoryMetadata(getColumns(), getIndices(), getConstraints());
}
void IStorage::alter(

View File

@ -242,11 +242,7 @@ MergeTreeData::MergeTreeData(
StorageInMemoryMetadata MergeTreeData::getInMemoryMetadata() const
{
StorageInMemoryMetadata metadata{
.columns = getColumns(),
.indices = getIndices(),
.constraints = getConstraints(),
};
StorageInMemoryMetadata metadata(getColumns(), getIndices(), getConstraints());
if (partition_by_ast)
metadata.partition_by_ast = partition_by_ast->clone();

View File

@ -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 " \
"before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS);
StorageInMemoryMetadata metadata{
.columns = args.columns,
.indices = indices_description,
.constraints = args.constraints,
.partition_by_ast = partition_by_ast,
.order_by_ast = order_by_ast,
.primary_key_ast = primary_key_ast,
.ttl_for_table_ast = ttl_table_ast,
.sample_by_ast = sample_by_ast,
.settings_ast = settings_ast,
};
StorageInMemoryMetadata metadata(args.columns, indices_description, args.constraints);
metadata.partition_by_ast = partition_by_ast;
metadata.order_by_ast = order_by_ast;
metadata.primary_key_ast = primary_key_ast;
metadata.ttl_for_table_ast = ttl_table_ast;
metadata.sample_by_ast = sample_by_ast;
metadata.settings_ast = settings_ast;
if (replicated)
return StorageReplicatedMergeTree::create(
zookeeper_path, replica_name, args.attach, args.table_id, args.relative_data_path,

View 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;
}
}

View File

@ -35,6 +35,12 @@ struct StorageInMemoryMetadata
ASTPtr settings_ast = nullptr;
/// SELECT QUERY. Supported for MaterializedView only.
ASTPtr select = nullptr;
StorageInMemoryMetadata(const StorageInMemoryMetadata & other);
StorageInMemoryMetadata() = default;
StorageInMemoryMetadata(const ColumnsDescription & columns_, const IndicesDescription & indices_, const ConstraintsDescription & constraints_);
StorageInMemoryMetadata & operator=(const StorageInMemoryMetadata & other);
};
}

View File

@ -165,13 +165,9 @@ bool StorageMaterializedView::hasColumn(const String & column_name) const
StorageInMemoryMetadata StorageMaterializedView::getInMemoryMetadata() const
{
return
{
.columns = getColumns(),
.indices = getIndices(),
.constraints = getConstraints(),
.select = getSelectQuery(),
};
StorageInMemoryMetadata result(getColumns(), getIndices(), getConstraints());
result.select = getSelectQuery();
return result;
}
QueryProcessingStage::Enum StorageMaterializedView::getQueryProcessingStage(const Context & context) const

View File

@ -411,7 +411,6 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes()
zookeeper->createIfNotExists(zookeeper_path + "/quorum", String());
zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", String());
zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", String());
zookeeper->createIfNotExists(zookeeper_path + "/alter_intention_counter", String());
/// Tracking lag of replicas.
zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", String());
@ -420,10 +419,6 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes()
/// Mutations
zookeeper->createIfNotExists(zookeeper_path + "/mutations", 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);
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();
if (columns_from_zk != old_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)
@ -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 + "/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 + "/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 + "/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.
@ -3255,6 +3253,7 @@ void StorageReplicatedMergeTree::alter(
StorageInMemoryMetadata metadata = getInMemoryMetadata();
params.apply(metadata);
changeSettings(metadata.settings_ast, table_lock_holder);
global_context.getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, metadata);

View File

@ -1,9 +1,9 @@
SET send_logs_level = 'none';
DROP TABLE IF EXISTS test.old_style;
CREATE TABLE test.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 }
DROP TABLE test.old_style;
DROP TABLE IF EXISTS old_style;
CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192);
ALTER TABLE old_style ADD COLUMN y UInt32, MODIFY ORDER BY (x, y); -- { serverError 36 }
DROP TABLE old_style;
DROP TABLE IF EXISTS test.summing_r1;
DROP TABLE IF EXISTS test.summing_r2;
@ -40,7 +40,7 @@ SELECT '*** Check SHOW CREATE TABLE ***';
SHOW CREATE 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;
SELECT '*** Check SHOW CREATE TABLE after offline ALTER ***';
SHOW CREATE TABLE test.summing_r2;