Revert "Add system.dropped_tables_parts table"

This commit is contained in:
Antonio Andelic 2023-12-19 10:36:22 +01:00 committed by GitHub
parent bfd403cc8c
commit 52b3caec90
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 64 additions and 225 deletions

View File

@ -1,14 +0,0 @@
---
slug: /en/operations/system-tables/dropped_tables_parts
---
# dropped_tables_parts {#system_tables-dropped_tables_parts}
Contains information about parts of [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) dropped tables from [system.dropped_tables](./dropped_tables.md)
The schema of this table is the same as [system.parts](./parts.md)
**See Also**
- [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md)
- [system.parts](./parts.md)
- [system.dropped_tables](./dropped_tables.md)

View File

@ -1,89 +0,0 @@
#include <Storages/StorageMaterializedMySQL.h>
#include <Storages/VirtualColumnUtils.h>
#include <Access/ContextAccess.h>
#include <Storages/System/StorageSystemDroppedTablesParts.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeUUID.h>
namespace DB
{
StoragesDroppedInfoStream::StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context)
: StoragesInfoStreamBase(context)
{
needsLock = false;
/// Will apply WHERE to subset of columns and then add more columns.
/// This is kind of complicated, but we use WHERE to do less work.
Block block_to_filter;
MutableColumnPtr database_column_mut = ColumnString::create();
MutableColumnPtr table_column_mut = ColumnString::create();
MutableColumnPtr engine_column_mut = ColumnString::create();
MutableColumnPtr active_column_mut = ColumnUInt8::create();
MutableColumnPtr storage_uuid_column_mut = ColumnUUID::create();
const auto access = context->getAccess();
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES);
auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped();
for (const auto & dropped_table : tables_mark_dropped)
{
StoragePtr storage = dropped_table.table;
if (!storage)
continue;
UUID storage_uuid = storage->getStorageID().uuid;
String database_name = storage->getStorageID().getDatabaseName();
String table_name = storage->getStorageID().getTableName();
String engine_name = storage->getName();
#if USE_MYSQL
if (auto * proxy = dynamic_cast<StorageMaterializedMySQL *>(storage.get()))
{
auto nested = proxy->getNested();
storage.swap(nested);
}
#endif
if (!dynamic_cast<MergeTreeData *>(storage.get()))
continue;
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
continue;
storages[storage_uuid] = storage;
/// Add all combinations of flag 'active'.
for (UInt64 active : {0, 1})
{
database_column_mut->insert(database_name);
table_column_mut->insert(table_name);
engine_column_mut->insert(engine_name);
active_column_mut->insert(active);
storage_uuid_column_mut->insert(storage_uuid);
}
}
block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared<DataTypeString>(), "database"));
block_to_filter.insert(ColumnWithTypeAndName(std::move(table_column_mut), std::make_shared<DataTypeString>(), "table"));
block_to_filter.insert(ColumnWithTypeAndName(std::move(engine_column_mut), std::make_shared<DataTypeString>(), "engine"));
block_to_filter.insert(ColumnWithTypeAndName(std::move(active_column_mut), std::make_shared<DataTypeUInt8>(), "active"));
block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared<DataTypeUUID>(), "uuid"));
if (block_to_filter.rows())
{
/// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'.
VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context);
rows = block_to_filter.rows();
}
database_column = block_to_filter.getByName("database").column;
table_column = block_to_filter.getByName("table").column;
active_column = block_to_filter.getByName("active").column;
storage_uuid_column = block_to_filter.getByName("uuid").column;
}
}

View File

@ -1,33 +0,0 @@
#pragma once
#include <Storages/System/StorageSystemParts.h>
namespace DB
{
class StoragesDroppedInfoStream : public StoragesInfoStreamBase
{
public:
StoragesDroppedInfoStream(const SelectQueryInfo & query_info, ContextPtr context);
};
class Context;
/** Implements system table 'dropped_tables_parts' which allows to get information about data parts for dropped but not yet removed tables.
*/
class StorageSystemDroppedTablesParts final : public StorageSystemParts
{
public:
explicit StorageSystemDroppedTablesParts(const StorageID & table_id) : StorageSystemParts(table_id) {}
std::string getName() const override { return "SystemDroppedTablesParts"; }
protected:
std::unique_ptr<StoragesInfoStreamBase> getStoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context) override
{
return std::make_unique<StoragesDroppedInfoStream>(query_info, context);
}
};
}

View File

@ -11,7 +11,7 @@ class Context;
/** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family.
*/
class StorageSystemParts : public StorageSystemPartsBase
class StorageSystemParts final : public StorageSystemPartsBase
{
public:
explicit StorageSystemParts(const StorageID & table_id_);

View File

@ -6,7 +6,6 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeUUID.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/StorageMaterializedMySQL.h>
#include <Storages/VirtualColumnUtils.h>
@ -23,6 +22,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
bool StorageSystemPartsBase::hasStateColumn(const Names & column_names, const StorageSnapshotPtr & storage_snapshot)
{
bool has_state_column = false;
@ -79,7 +83,7 @@ StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, boo
}
StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context)
: StoragesInfoStreamBase(context)
: query_id(context->getCurrentQueryId()), settings(context->getSettingsRef())
{
/// Will apply WHERE to subset of columns and then add more columns.
/// This is kind of complicated, but we use WHERE to do less work.
@ -89,7 +93,6 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte
MutableColumnPtr table_column_mut = ColumnString::create();
MutableColumnPtr engine_column_mut = ColumnString::create();
MutableColumnPtr active_column_mut = ColumnUInt8::create();
MutableColumnPtr storage_uuid_column_mut = ColumnUUID::create();
const auto access = context->getAccess();
const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES);
@ -136,7 +139,6 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte
continue;
String engine_name = storage->getName();
UUID storage_uuid = storage->getStorageID().uuid;
#if USE_MYSQL
if (auto * proxy = dynamic_cast<StorageMaterializedMySQL *>(storage.get()))
@ -151,7 +153,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
continue;
storages[storage_uuid] = storage;
storages[std::make_pair(database_name, iterator->name())] = storage;
/// Add all combinations of flag 'active'.
for (UInt64 active : {0, 1})
@ -159,7 +161,6 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte
table_column_mut->insert(table_name);
engine_column_mut->insert(engine_name);
active_column_mut->insert(active);
storage_uuid_column_mut->insert(storage_uuid);
}
offsets[i] += 2;
@ -177,7 +178,6 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte
block_to_filter.insert(ColumnWithTypeAndName(std::move(table_column_mut), std::make_shared<DataTypeString>(), "table"));
block_to_filter.insert(ColumnWithTypeAndName(std::move(engine_column_mut), std::make_shared<DataTypeString>(), "engine"));
block_to_filter.insert(ColumnWithTypeAndName(std::move(active_column_mut), std::make_shared<DataTypeUInt8>(), "active"));
block_to_filter.insert(ColumnWithTypeAndName(std::move(storage_uuid_column_mut), std::make_shared<DataTypeUUID>(), "uuid"));
if (rows)
{
@ -189,9 +189,57 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, Conte
database_column = block_to_filter.getByName("database").column;
table_column = block_to_filter.getByName("table").column;
active_column = block_to_filter.getByName("active").column;
storage_uuid_column = block_to_filter.getByName("uuid").column;
next_row = 0;
}
StoragesInfo StoragesInfoStream::next()
{
while (next_row < rows)
{
StoragesInfo info;
info.database = (*database_column)[next_row].get<String>();
info.table = (*table_column)[next_row].get<String>();
auto is_same_table = [&info, this] (size_t row) -> bool
{
return (*database_column)[row].get<String>() == info.database &&
(*table_column)[row].get<String>() == info.table;
};
/// We may have two rows per table which differ in 'active' value.
/// If rows with 'active = 0' were not filtered out, this means we
/// must collect the inactive parts. Remember this fact in StoragesInfo.
for (; next_row < rows && is_same_table(next_row); ++next_row)
{
const auto active = (*active_column)[next_row].get<UInt64>();
if (active == 0)
info.need_inactive_parts = true;
}
info.storage = storages.at(std::make_pair(info.database, info.table));
/// For table not to be dropped and set of columns to remain constant.
info.table_lock = info.storage->tryLockForShare(query_id, settings.lock_acquire_timeout);
if (info.table_lock == nullptr)
{
// Table was dropped while acquiring the lock, skipping table
continue;
}
info.engine = info.storage->getName();
info.data = dynamic_cast<MergeTreeData *>(info.storage.get());
if (!info.data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown engine {}", info.engine);
return info;
}
return {};
}
Pipe StorageSystemPartsBase::read(
const Names & column_names,
@ -204,7 +252,7 @@ Pipe StorageSystemPartsBase::read(
{
bool has_state_column = hasStateColumn(column_names, storage_snapshot);
auto stream = getStoragesInfoStream(query_info, context);
StoragesInfoStream stream(query_info, context);
/// Create the result.
Block sample = storage_snapshot->metadata->getSampleBlock();
@ -215,7 +263,7 @@ Pipe StorageSystemPartsBase::read(
if (has_state_column)
res_columns.push_back(ColumnString::create());
while (StoragesInfo info = stream->next())
while (StoragesInfo info = stream.next())
{
processNextStorage(context, res_columns, columns_mask, info, has_state_column);
}

View File

@ -8,11 +8,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class Context;
struct StoragesInfo
@ -34,64 +29,13 @@ struct StoragesInfo
};
/** A helper class that enumerates the storages that match given query. */
class StoragesInfoStreamBase
class StoragesInfoStream
{
public:
StoragesInfoStreamBase(ContextPtr context)
: query_id(context->getCurrentQueryId()), settings(context->getSettingsRef()), next_row(0), rows(0)
{}
StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context);
StoragesInfo next();
StoragesInfo next()
{
while (next_row < rows)
{
StoragesInfo info;
info.database = (*database_column)[next_row].get<String>();
info.table = (*table_column)[next_row].get<String>();
UUID storage_uuid = (*storage_uuid_column)[next_row].get<UUID>();
auto is_same_table = [&storage_uuid, this] (size_t row) -> bool
{
return (*storage_uuid_column)[row].get<UUID>() == storage_uuid;
};
/// We may have two rows per table which differ in 'active' value.
/// If rows with 'active = 0' were not filtered out, this means we
/// must collect the inactive parts. Remember this fact in StoragesInfo.
for (; next_row < rows && is_same_table(next_row); ++next_row)
{
const auto active = (*active_column)[next_row].get<UInt64>();
if (active == 0)
info.need_inactive_parts = true;
}
info.storage = storages.at(storage_uuid);
if (needsLock)
{
/// For table not to be dropped and set of columns to remain constant.
info.table_lock = info.storage->tryLockForShare(query_id, settings.lock_acquire_timeout);
if (info.table_lock == nullptr)
{
// Table was dropped while acquiring the lock, skipping table
continue;
}
}
info.engine = info.storage->getName();
info.data = dynamic_cast<MergeTreeData *>(info.storage.get());
if (!info.data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown engine {}", info.engine);
return info;
}
return {};
}
protected:
private:
String query_id;
Settings settings;
@ -99,22 +43,12 @@ protected:
ColumnPtr database_column;
ColumnPtr table_column;
ColumnPtr active_column;
ColumnPtr storage_uuid_column;
size_t next_row;
size_t rows;
using StoragesMap = std::unordered_map<UUID, StoragePtr>;
using StoragesMap = std::map<std::pair<String, String>, StoragePtr>;
StoragesMap storages;
bool needsLock = true;
};
class StoragesInfoStream : public StoragesInfoStreamBase
{
public:
StoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context);
};
/** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family.
@ -143,11 +77,6 @@ protected:
StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_);
virtual std::unique_ptr<StoragesInfoStreamBase> getStoragesInfoStream(const SelectQueryInfo & query_info, ContextPtr context)
{
return std::make_unique<StoragesInfoStream>(query_info, context);
}
virtual void
processNextStorage(ContextPtr context, MutableColumns & columns, std::vector<UInt8> & columns_mask, const StoragesInfo & info, bool has_state_column) = 0;
};

View File

@ -82,7 +82,6 @@
#include <Storages/System/StorageSystemCertificates.h>
#include <Storages/System/StorageSystemSchemaInferenceCache.h>
#include <Storages/System/StorageSystemDroppedTables.h>
#include <Storages/System/StorageSystemDroppedTablesParts.h>
#include <Storages/System/StorageSystemZooKeeperConnection.h>
#include <Storages/System/StorageSystemJemalloc.h>
#include <Storages/System/StorageSystemScheduler.h>
@ -157,7 +156,6 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
attach<StorageSystemBackups>(context, system_database, "backups");
attach<StorageSystemSchemaInferenceCache>(context, system_database, "schema_inference_cache");
attach<StorageSystemDroppedTables>(context, system_database, "dropped_tables");
attach<StorageSystemDroppedTablesParts>(context, system_database, "dropped_tables_parts");
attach<StorageSystemScheduler>(context, system_database, "scheduler");
#if defined(__ELF__) && !defined(OS_FREEBSD)
attach<StorageSystemSymbols>(context, system_database, "symbols");