提交 f19bb041 编写于 作者: A Alexander Tokmakov

add asserts on invalid StorageID

上级 250f170a
......@@ -119,8 +119,8 @@ namespace ErrorCodes
/// Assumes `storage` is set and the table filter (row-level security) is not empty.
String InterpreterSelectQuery::generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const
{
const auto & db_name = table_id.value().database_name;
const auto & table_name = table_id.value().table_name;
const auto & db_name = table_id.getDatabaseName();
const auto & table_name = table_id.getTableName();
/// TODO: implement some AST builders for this kind of stuff
ASTPtr query_ast = std::make_shared<ASTSelectQuery>();
......@@ -369,7 +369,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
source_header = storage->getSampleBlockForColumns(required_columns);
/// Fix source_header for filter actions.
auto row_policy_filter = context->getRowPolicy()->getCondition(table_id->database_name, table_id->table_name, RowPolicy::SELECT_FILTER);
auto row_policy_filter = context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
if (row_policy_filter)
{
filter_info = std::make_shared<FilterInfo>();
......@@ -413,9 +413,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
sanitizeBlock(result_header);
/// Remove limits for some tables in the `system` database.
if (storage && (table_id->database_name == "system"))
if (storage && (table_id.getDatabaseName() == "system"))
{
String table_name = table_id->table_name;
String table_name = table_id.getTableName();
if ((table_name == "quotas") || (table_name == "quota_usage") || (table_name == "one"))
{
options.ignore_quota = true;
......@@ -494,7 +494,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
/// PREWHERE optimization.
/// Turn off, if the table filter (row-level security) is applied.
if (storage && !context->getRowPolicy()->getCondition(table_id->table_name, table_id->database_name, RowPolicy::SELECT_FILTER))
if (storage && !context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER))
{
query_analyzer->makeSetsForIndex(query.where());
query_analyzer->makeSetsForIndex(query.prewhere());
......@@ -1355,7 +1355,7 @@ void InterpreterSelectQuery::executeFetchColumns(
if (storage)
{
/// Append columns from the table filter to required
auto row_policy_filter = context->getRowPolicy()->getCondition(table_id->database_name, table_id->table_name, RowPolicy::SELECT_FILTER);
auto row_policy_filter = context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER);
if (row_policy_filter)
{
auto initial_required_columns = required_columns;
......
......@@ -296,7 +296,7 @@ private:
/// Table from where to read data, if not subquery.
StoragePtr storage;
std::optional<StorageID> table_id;
StorageID table_id = StorageID::createEmpty(); /// Will be initialized if storage is not nullptr
TableStructureReadLockHolder table_lock;
/// Used when we read from prepared input, not table or subquery.
......
......@@ -139,7 +139,7 @@ public:
const Context & context);
private:
StorageID select_table_id;
StorageID select_table_id = StorageID::createEmpty(); /// Will be initialized in constructor
ASTPtr inner_query;
Context & global_context;
bool is_temporary = false;
......
......@@ -19,31 +19,53 @@ struct StorageID
{
String database_name;
String table_name;
UUID uuid;
UUID uuid = UUID{UInt128(0, 0)};
//StorageID() = delete;
StorageID() = default;
StorageID(const String & database, const String & table, UUID uuid_ = UUID{UInt128(0, 0)})
: database_name(database), table_name(table), uuid(uuid_)
{
assertNotEmpty();
}
String getDatabaseName() const
{
assertNotEmpty();
return database_name;
}
String getTableName() const
{
assertNotEmpty();
return table_name;
}
String getFullTableName() const
{
assert_valid();
assertNotEmpty();
return (database_name.empty() ? "" : database_name + ".") + table_name;
}
String getNameForLogs() const
{
assert_valid();
return (database_name.empty() ? "" : backQuoteIfNeed(database_name) + ".") + backQuoteIfNeed(table_name) + (hasUUID() ? "" : " (UUID " + toString(uuid) + ")");
assertNotEmpty();
return (database_name.empty() ? "" : backQuoteIfNeed(database_name) + ".") + backQuoteIfNeed(table_name)
+ (hasUUID() ? "" : " (UUID " + toString(uuid) + ")");
}
bool empty() const
{
return table_name.empty() && !hasUUID();
}
bool hasUUID() const
{
return uuid != UUID{UInt128(0, 0)};
}
bool operator<(const StorageID & rhs) const
{
assert_valid();
assertNotEmpty();
/// It's needed for ViewDependencies
if (!hasUUID() && !rhs.hasUUID())
/// If both IDs don't have UUID, compare them like pair of strings
......@@ -56,24 +78,21 @@ struct StorageID
return !hasUUID();
}
bool empty() const
{
return table_name.empty() || (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && !hasUUID());
}
void assert_valid() const
void assertNotEmpty() const
{
if (empty())
throw Exception("empty table name", ErrorCodes::LOGICAL_ERROR);
if (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && !hasUUID() && !database_name.empty())
throw Exception("unexpected database name", ErrorCodes::LOGICAL_ERROR);
throw Exception("Both table name and UUID are empty", ErrorCodes::LOGICAL_ERROR);
if (table_name == TABLE_WITH_UUID_NAME_PLACEHOLDER && !hasUUID())
throw Exception("Table name was replaced with placeholder, but UUID is Nil", ErrorCodes::LOGICAL_ERROR);
if (table_name.empty() && !database_name.empty())
throw Exception("Table name is empty, but database name is not", ErrorCodes::LOGICAL_ERROR);
}
bool hasUUID() const
{
return uuid != UUID{UInt128(0, 0)};
}
/// Avoid implicit construction of empty StorageID. However, it's needed for deferred initialization.
static StorageID createEmpty() { return {}; }
private:
StorageID() = default;
};
}
......@@ -36,25 +36,25 @@ static inline String generateInnerTableName(const String & table_name)
return ".inner." + table_name;
}
StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context & context, bool is_live_view /*= false*/, bool need_visitor /*= true*/)
StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context & context, bool is_live_view /*= false*/, bool add_default_db /*= true*/)
{
if (need_visitor)
if (add_default_db)
{
AddDefaultDatabaseVisitor visitor(context.getCurrentDatabase(), nullptr);
visitor.visit(query);
}
auto db_and_table = getDatabaseAndTable(query, 0);
ASTPtr subquery = extractTableExpression(query, 0);
if (!db_and_table && !subquery)
return {};
if (db_and_table)
if (auto db_and_table = getDatabaseAndTable(query, 0))
{
return StorageID(db_and_table->database, db_and_table->table/*, db_and_table->uuid*/);
}
else if (auto * ast_select = subquery->as<ASTSelectWithUnionQuery>())
else if (auto subquery = extractTableExpression(query, 0))
{
auto * ast_select = subquery->as<ASTSelectWithUnionQuery>();
if (!ast_select)
throw Exception(String("Logical error while creating Storage") + (is_live_view ? "Live" : "Materialized") +
"View. Could not retrieve table name from select query.",
DB::ErrorCodes::LOGICAL_ERROR);
if (ast_select->list_of_selects->children.size() != 1)
throw Exception(String("UNION is not supported for ") + (is_live_view ? "LIVE VIEW" : "MATERIALIZED VIEW"),
is_live_view ? ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW : ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW);
......@@ -64,9 +64,7 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context &
return extractDependentTableFromSelectQuery(inner_query->as<ASTSelectQuery &>(), context, is_live_view, false);
}
else
throw Exception(String("Logical error while creating Storage") + (is_live_view ? "Live" : "Materialized") +
"View. Could not retrieve table name from select query.",
DB::ErrorCodes::LOGICAL_ERROR);
return StorageID::createEmpty();
}
......@@ -214,7 +212,8 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context,
void StorageMaterializedView::drop(TableStructureWriteLockHolder &)
{
auto table_id = getStorageID();
global_context.removeDependency(select_table_id, table_id);
if (!select_table_id.empty())
global_context.removeDependency(select_table_id, table_id);
if (has_inner_table && tryGetTargetTable())
executeDropQuery(ASTDropQuery::Kind::Drop, global_context, target_table_id);
......@@ -278,16 +277,18 @@ void StorageMaterializedView::rename(
}
auto lock = global_context.getLock();
global_context.removeDependencyUnsafe(select_table_id, getStorageID());
if (!select_table_id.empty())
global_context.removeDependencyUnsafe(select_table_id, getStorageID());
IStorage::renameInMemory(new_database_name, new_table_name);
global_context.addDependencyUnsafe(select_table_id, getStorageID());
if (!select_table_id.empty())
global_context.addDependencyUnsafe(select_table_id, getStorageID());
}
void StorageMaterializedView::shutdown()
{
auto table_id = getStorageID();
/// Make sure the dependency is removed after DETACH TABLE
global_context.removeDependency(select_table_id, table_id);
if (!select_table_id.empty())
global_context.removeDependency(select_table_id, getStorageID());
}
StoragePtr StorageMaterializedView::getTargetTable() const
......
......@@ -9,7 +9,7 @@
namespace DB
{
StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context & context, bool is_live_view = false, bool need_visitor = true);
StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, Context & context, bool is_live_view = false, bool add_default_db = true);
class StorageMaterializedView : public ext::shared_ptr_helper<StorageMaterializedView>, public IStorage
......@@ -69,8 +69,11 @@ public:
Strings getDataPaths() const override;
private:
StorageID select_table_id;
StorageID target_table_id;
/// Can be empty if SELECT query doesn't contain table
StorageID select_table_id = StorageID::createEmpty();
/// Will be initialized in constructor
StorageID target_table_id = StorageID::createEmpty();
ASTPtr inner_query;
Context & global_context;
bool has_inner_table = false;
......
......@@ -117,7 +117,7 @@ bool StorageMerge::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, cons
size_t i = 0;
for (const auto & table : selected_tables)
{
if (table.first->mayBenefitFromIndexForIn(left_in_operand, query_context))
if (std::get<0>(table)->mayBenefitFromIndexForIn(left_in_operand, query_context))
return true;
++i;
......@@ -189,12 +189,12 @@ BlockInputStreams StorageMerge::read(
* This is necessary to correctly pass the recommended number of threads to each table.
*/
StorageListWithLocks selected_tables = getSelectedTables(
query_info.query, has_table_virtual_column, true, context.getCurrentQueryId());
query_info.query, has_table_virtual_column, context.getCurrentQueryId());
if (selected_tables.empty())
/// FIXME: do we support sampling in this case?
return createSourceStreams(
query_info, processed_stage, max_block_size, header, {}, {}, real_column_names, modified_context, 0, has_table_virtual_column);
query_info, processed_stage, max_block_size, header, {}, real_column_names, modified_context, 0, has_table_virtual_column);
size_t tables_count = selected_tables.size();
Float64 num_streams_multiplier = std::min(unsigned(tables_count), std::max(1U, unsigned(context.getSettingsRef().max_streams_multiplier_for_merge_tables)));
......@@ -206,7 +206,7 @@ BlockInputStreams StorageMerge::read(
{
for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it)
{
auto current_info = query_info.order_by_optimizer->getInputOrder(it->first);
auto current_info = query_info.order_by_optimizer->getInputOrder(std::get<0>(*it));
if (it == selected_tables.begin())
input_sorting_info = current_info;
else if (!current_info || (input_sorting_info && *current_info != *input_sorting_info))
......@@ -226,8 +226,7 @@ BlockInputStreams StorageMerge::read(
remaining_streams -= current_streams;
current_streams = std::max(size_t(1), current_streams);
StoragePtr storage = it->first;
TableStructureReadLockHolder struct_lock = it->second;
auto & storage = std::get<0>(*it);
/// If sampling requested, then check that table supports it.
if (query_info.query->as<ASTSelectQuery>()->sample_size() && !storage->supportsSampling())
......@@ -238,8 +237,8 @@ BlockInputStreams StorageMerge::read(
if (current_streams)
{
source_streams = createSourceStreams(
query_info, processed_stage, max_block_size, header, storage,
struct_lock, real_column_names, modified_context, current_streams, has_table_virtual_column);
query_info, processed_stage, max_block_size, header, *it, real_column_names, modified_context,
current_streams, has_table_virtual_column);
}
else
{
......@@ -247,7 +246,7 @@ BlockInputStreams StorageMerge::read(
header, [=]() mutable -> BlockInputStreamPtr
{
BlockInputStreams streams = createSourceStreams(query_info, processed_stage, max_block_size,
header, storage, struct_lock, real_column_names,
header, *it, real_column_names,
modified_context, current_streams, has_table_virtual_column, true);
if (!streams.empty() && streams.size() != 1)
......@@ -268,15 +267,15 @@ BlockInputStreams StorageMerge::read(
}
BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage,
const UInt64 max_block_size, const Block & header, const StoragePtr & storage,
const TableStructureReadLockHolder & struct_lock, Names & real_column_names,
const UInt64 max_block_size, const Block & header, const StorageWithLockAndName & storage_with_lock,
Names & real_column_names,
Context & modified_context, size_t streams_num, bool has_table_virtual_column,
bool concat_streams)
{
auto & [storage, struct_lock, table_name] = storage_with_lock;
SelectQueryInfo modified_query_info = query_info;
modified_query_info.query = query_info.query->clone();
String table_name = storage ? storage->getStorageID().table_name : "";
VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name);
if (!storage)
......@@ -289,7 +288,7 @@ BlockInputStreams StorageMerge::createSourceStreams(const SelectQueryInfo & quer
if (processed_stage <= storage->getQueryProcessingStage(modified_context))
{
/// If there are only virtual columns in query, you must request at least one other column.
if (real_column_names.size() ==0)
if (real_column_names.empty())
real_column_names.push_back(ExpressionActions::getSmallestColumn(storage->getColumns().getAllPhysical()));
source_streams = storage->read(real_column_names, modified_query_info, modified_context, processed_stage, max_block_size,
......@@ -360,7 +359,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String
}
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr & query, bool has_virtual_column, bool get_lock, const String & query_id) const
StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr & query, bool has_virtual_column, const String & query_id) const
{
StorageListWithLocks selected_tables;
DatabaseTablesIteratorPtr iterator = getDatabaseIterator(global_context);
......@@ -376,9 +375,8 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
if (storage.get() != this)
{
selected_tables.emplace_back(storage, get_lock ? storage->lockStructureForShare(false, query_id) : TableStructureReadLockHolder{});
auto table_id = storage->getStorageID();
virtual_column->insert(table_id.table_name);
selected_tables.emplace_back(storage, storage->lockStructureForShare(false, query_id), iterator->name());
virtual_column->insert(iterator->name());
}
iterator->next();
......@@ -391,8 +389,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
auto values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_table");
/// Remove unused tables from the list
//FIXME table name can be changed, use StorageID
selected_tables.remove_if([&] (const auto & elem) { return values.find(elem.first->getStorageID().table_name) == values.end(); });
selected_tables.remove_if([&] (const auto & elem) { return values.find(std::get<2>(elem)) == values.end(); });
}
return selected_tables;
......
......@@ -53,11 +53,12 @@ private:
OptimizedRegularExpression table_name_regexp;
Context global_context;
using StorageListWithLocks = std::list<std::pair<StoragePtr, TableStructureReadLockHolder>>;
using StorageWithLockAndName = std::tuple<StoragePtr, TableStructureReadLockHolder, String>;
using StorageListWithLocks = std::list<StorageWithLockAndName>;
StorageListWithLocks getSelectedTables(const String & query_id) const;
StorageMerge::StorageListWithLocks getSelectedTables(const ASTPtr & query, bool has_virtual_column, bool get_lock, const String & query_id) const;
StorageMerge::StorageListWithLocks getSelectedTables(const ASTPtr & query, bool has_virtual_column, const String & query_id) const;
template <typename F>
StoragePtr getFirstTable(F && predicate) const;
......@@ -76,8 +77,8 @@ protected:
const Context & context, QueryProcessingStage::Enum processed_stage);
BlockInputStreams createSourceStreams(const SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage,
const UInt64 max_block_size, const Block & header, const StoragePtr & storage,
const TableStructureReadLockHolder & struct_lock, Names & real_column_names,
const UInt64 max_block_size, const Block & header, const StorageWithLockAndName & storage_with_lock,
Names & real_column_names,
Context & modified_context, size_t streams_num, bool has_table_virtual_column,
bool concat_streams = false);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册