Merge branch 'yurial-change-shared-ptr-helper'

This commit is contained in:
Alexey Milovidov 2017-11-04 19:46:37 +03:00
commit fbb8f48325
11 changed files with 19 additions and 19 deletions

View File

@ -149,8 +149,8 @@ void AsynchronousMetrics::update()
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
{
auto & table = iterator->table();
StorageMergeTree * table_merge_tree = typeid_cast<StorageMergeTree *>(table.get());
StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast<StorageReplicatedMergeTree *>(table.get());
StorageMergeTree * table_merge_tree = dynamic_cast<StorageMergeTree *>(table.get());
StorageReplicatedMergeTree * table_replicated_merge_tree = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
if (table_replicated_merge_tree)
{

View File

@ -1584,7 +1584,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
if (table)
{
StorageSet * storage_set = typeid_cast<StorageSet *>(table.get());
StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get());
if (storage_set)
{
@ -2416,7 +2416,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
if (table)
{
StorageJoin * storage_join = typeid_cast<StorageJoin *>(table.get());
StorageJoin * storage_join = dynamic_cast<StorageJoin *>(table.get());
if (storage_join)
{

View File

@ -210,7 +210,7 @@ void InJoinSubqueriesPreprocessor::process(ASTSelectQuery * query) const
bool InJoinSubqueriesPreprocessor::hasAtLeastTwoShards(const IStorage & table) const
{
const StorageDistributed * distributed = typeid_cast<const StorageDistributed *>(&table);
const StorageDistributed * distributed = dynamic_cast<const StorageDistributed *>(&table);
if (!distributed)
return false;
@ -221,7 +221,7 @@ bool InJoinSubqueriesPreprocessor::hasAtLeastTwoShards(const IStorage & table) c
std::pair<std::string, std::string>
InJoinSubqueriesPreprocessor::getRemoteDatabaseAndTableName(const IStorage & table) const
{
const StorageDistributed & distributed = typeid_cast<const StorageDistributed &>(table);
const StorageDistributed & distributed = dynamic_cast<const StorageDistributed &>(table);
return { distributed.getRemoteDatabaseName(), distributed.getRemoteTableName() };
}

View File

@ -144,7 +144,7 @@ BlockIO InterpreterCheckQuery::execute()
StoragePtr table = context.getTable(database_name, table_name);
auto distributed_table = typeid_cast<StorageDistributed *>(&*table);
auto distributed_table = dynamic_cast<StorageDistributed *>(&*table);
if (distributed_table != nullptr)
{
/// For tables with the Distributed engine, the CHECK TABLE query sends a DESCRIBE TABLE request to all replicas.

View File

@ -839,9 +839,9 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
MergeTreeWhereOptimizer{query_info, context, merge_tree.getData(), required_columns, log};
};
if (const StorageMergeTree * merge_tree = typeid_cast<const StorageMergeTree *>(storage.get()))
if (const StorageMergeTree * merge_tree = dynamic_cast<const StorageMergeTree *>(storage.get()))
optimize_prewhere(*merge_tree);
else if (const StorageReplicatedMergeTree * merge_tree = typeid_cast<const StorageReplicatedMergeTree *>(storage.get()))
else if (const StorageReplicatedMergeTree * merge_tree = dynamic_cast<const StorageReplicatedMergeTree *>(storage.get()))
optimize_prewhere(*merge_tree);
}

View File

@ -64,7 +64,7 @@ public:
if (!table.isRemote())
return false;
const StorageDistributedFake * distributed = typeid_cast<const StorageDistributedFake *>(&table);
const StorageDistributedFake * distributed = dynamic_cast<const StorageDistributedFake *>(&table);
if (!distributed)
return false;
@ -74,7 +74,7 @@ public:
std::pair<std::string, std::string>
getRemoteDatabaseAndTableName(const DB::IStorage & table) const override
{
const StorageDistributedFake & distributed = typeid_cast<const StorageDistributedFake &>(table);
const StorageDistributedFake & distributed = dynamic_cast<const StorageDistributedFake &>(table);
return { distributed.getRemoteDatabaseName(), distributed.getRemoteTableName() };
}
};

View File

@ -43,7 +43,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
{
auto & table = iterator->table();
StorageReplicatedMergeTree * table_replicated = typeid_cast<StorageReplicatedMergeTree *>(table.get());
StorageReplicatedMergeTree * table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
if (!table_replicated)
continue;

View File

@ -145,12 +145,12 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & body
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ABORTED && e.code() != ErrorCodes::CANNOT_WRITE_TO_OSTREAM)
typeid_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
dynamic_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
throw;
}
catch (...)
{
typeid_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
dynamic_cast<StorageReplicatedMergeTree &>(*owned_storage).enqueuePartForCheck(part_name);
throw;
}
}

View File

@ -592,7 +592,7 @@ void ReshardingWorker::perform(const std::string & job_descriptor, const std::st
throw Exception{"Coordinator has been deleted", ErrorCodes::RESHARDING_COORDINATOR_DELETED};
StoragePtr generic_storage = context.getTable(current_job.database_name, current_job.table_name);
auto & storage = typeid_cast<StorageReplicatedMergeTree &>(*generic_storage);
auto & storage = dynamic_cast<StorageReplicatedMergeTree &>(*generic_storage);
current_job.storage = &storage;
std::string dumped_coordinator_state;

View File

@ -63,7 +63,7 @@ BlockInputStreams StorageSystemReplicas::read(
std::map<String, std::map<String, StoragePtr>> replicated_tables;
for (const auto & db : context.getDatabases())
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
if (typeid_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
replicated_tables[db.first][iterator->name()] = iterator->table();
/// Do you need columns that require a walkthrough in ZooKeeper to compute.
@ -135,7 +135,7 @@ BlockInputStreams StorageSystemReplicas::read(
for (size_t i = 0, size = col_database.column->size(); i < size; ++i)
{
StorageReplicatedMergeTree::Status status;
typeid_cast<StorageReplicatedMergeTree &>(
dynamic_cast<StorageReplicatedMergeTree &>(
*replicated_tables
[(*col_database.column)[i].safeGet<const String &>()]
[(*col_table.column)[i].safeGet<const String &>()]).getStatus(status, with_zk_fields);

View File

@ -61,7 +61,7 @@ BlockInputStreams StorageSystemReplicationQueue::read(
std::map<String, std::map<String, StoragePtr>> replicated_tables;
for (const auto & db : context.getDatabases())
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
if (typeid_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
replicated_tables[db.first][iterator->name()] = iterator->table();
ColumnWithTypeAndName col_database_to_filter { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "database" };
@ -118,7 +118,7 @@ BlockInputStreams StorageSystemReplicationQueue::read(
String database = (*col_database_to_filter.column)[i].safeGet<const String &>();
String table = (*col_table_to_filter.column)[i].safeGet<const String &>();
typeid_cast<StorageReplicatedMergeTree &>(*replicated_tables[database][table]).getQueue(queue, replica_name);
dynamic_cast<StorageReplicatedMergeTree &>(*replicated_tables[database][table]).getQueue(queue, replica_name);
for (size_t j = 0, queue_size = queue.size(); j < queue_size; ++j)
{