Skip to content

Commit

Permalink
improve TableNameHints
Browse files Browse the repository at this point in the history
  • Loading branch information
bharatnc committed Dec 2, 2023
1 parent 56366b7 commit a47b2c1
Show file tree
Hide file tree
Showing 4 changed files with 78 additions and 62 deletions.
29 changes: 6 additions & 23 deletions src/Databases/IDatabase.cpp
Expand Up @@ -21,38 +21,21 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const
{
if (auto storage = tryGetTable(name, context))
return storage;
TableNameHints hints(this->shared_from_this(), context);
std::vector<String> names = hints.getHints(name);
if (names.empty())
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name));
else
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist. Maybe you meant {}?", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name), backQuoteIfNeed(names[0]));
}

StoragePtr IDatabase::getTableAcrossAllDatabases(const String & name, ContextPtr context, Databases databases) const
{
if (auto storage = tryGetTable(name, context))
return storage;

std::vector<std::pair<std::string, std::string>> db_and_table_names;
for (const auto & db : databases)
{
TableNameHints hints(db.second, context);
auto table_names = hints.getHints(name);
for (const auto & table_name : table_names)
db_and_table_names.emplace_back(std::pair(db.second->getDatabaseName(), table_name));
}
TableNameHints hints(this->shared_from_this(), context);
/// hint is a pair which holds a single database_name and table_name suggestion for the given table name.
auto hint = hints.getHintForTable(name);

if (db_and_table_names.empty())
if (hint.first.empty())
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name));
else
throw Exception(
ErrorCodes::UNKNOWN_TABLE,
"Table {}.{} does not exist. Maybe you meant {}.{}?",
backQuoteIfNeed(getDatabaseName()),
backQuoteIfNeed(name),
backQuoteIfNeed(db_and_table_names[0].first),
backQuoteIfNeed(db_and_table_names[0].second));
backQuoteIfNeed(hint.first),
backQuoteIfNeed(hint.second));
}

std::vector<std::pair<ASTPtr, StoragePtr>> IDatabase::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const
Expand Down
8 changes: 0 additions & 8 deletions src/Databases/IDatabase.h
Expand Up @@ -170,15 +170,7 @@ class IDatabase : public std::enable_shared_from_this<IDatabase>
/// Get the table for work. Return nullptr if there is no table.
virtual StoragePtr tryGetTable(const String & name, ContextPtr context) const = 0;

using DatabasePtr = std::shared_ptr<IDatabase>;
using Databases = std::map<String, DatabasePtr>;

virtual StoragePtr getTable(const String & name, ContextPtr context) const;
/// Similar to getTable and tries to get the table from the the database in context. Except, if the table is
/// not found, it tries search for the table across all the current databases in the database catalog and returns
/// that in the hint as database.table.
virtual StoragePtr getTableAcrossAllDatabases(const String & name, ContextPtr context, Databases databases) const;


virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; }

Expand Down
9 changes: 1 addition & 8 deletions src/Interpreters/DatabaseCatalog.cpp
Expand Up @@ -423,14 +423,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
{
try
{
Databases current_databases;
{
std::lock_guard lock(databases_mutex);
current_databases = databases;
}
/// getTableAcrossAllDatabases returns the table if found in the current context. Otherwise, it tries to search for that
/// table across all the current databases. If found, it adds the database.table to the hint thrown in the exception.
table = database->getTableAcrossAllDatabases(table_id.table_name, context_, current_databases);
table = database->getTable(table_id.table_name, context_);
}
catch (const Exception & e)
{
Expand Down
94 changes: 71 additions & 23 deletions src/Interpreters/DatabaseCatalog.h
Expand Up @@ -2,6 +2,7 @@

#include <Core/UUID.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/StorageID.h>
#include <Databases/TablesDependencyGraph.h>
#include <Parsers/IAST_fwd.h>
Expand Down Expand Up @@ -30,29 +31,6 @@ namespace fs = std::filesystem;
namespace DB
{

class TableNameHints : public IHints<>
{
public:
TableNameHints(ConstDatabasePtr database_, ContextPtr context_)
: context(context_),
database(database_)
{
}
Names getAllRegisteredNames() const override
{
Names result;
if (database)
{
for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next())
result.emplace_back(table_it->name());
}
return result;
}
private:
ContextPtr context;
ConstDatabasePtr database;
};

class IDatabase;
class Exception;
class ColumnsDescription;
Expand Down Expand Up @@ -392,6 +370,76 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext
static constexpr time_t DBMS_DEFAULT_DISK_RELOAD_PERIOD_SEC = 5;
};

class TableNameHints : public IHints<>
{
public:
TableNameHints(ConstDatabasePtr database_, ContextPtr context_)
: context(context_),
database(database_)
{
}

/// getHintForTable tries to get a hint for the provided table_name in the provided
/// database. If the results are empty, it goes for extended hints for the table
/// with getExtendedHintForTable which looks for the table name in every database that's
/// available in the database catalog. It finally returns a single hint which is the database
/// name and table_name pair which is similar to the table_name provided. Perhaps something to
/// consider is should we return more than one pair of hint?
std::pair<String, String> getHintForTable(const String & table_name) const
{
auto results = this->getHints(table_name, getAllRegisteredNames());
if (results.empty())
return getExtendedHintForTable(table_name);
return std::make_pair(database->getDatabaseName(), results[0]);
}

/// getExtendedHintsForTable tries to get hint for the given table_name across all
/// the databases that are available in the database catalog.
std::pair<String, String> getExtendedHintForTable(const String & table_name) const
{
/// load all available databases from the DatabaseCatalog instance
auto & database_catalog = DatabaseCatalog::instance();
auto all_databases = database_catalog.getDatabases();

/// For returning a pair of hint, which will be the database_name and table_name
/// that's closest to the given table_name.
std::pair<String, String> result;

for (const auto & [db_name, db] : all_databases)
{
/// this case should be covered already by getHintForTable
if (db_name == database->getDatabaseName())
continue;

TableNameHints hints(db, context);
auto results = hints.getHints(table_name);

/// if results are empty continue, else, break and return the first instance of
/// table_name and database_name that's found.
if (results.empty())
continue;

result = std::make_pair(db_name, results[0]);
break;
}

return result;
}

Names getAllRegisteredNames() const override
{
Names result;
if (database)
for (auto table_it = database->getTablesIterator(context); table_it->isValid(); table_it->next())
result.emplace_back(table_it->name());
return result;
}

private:
ContextPtr context;
ConstDatabasePtr database;
};


/// This class is useful when creating a table or database.
/// Usually we create IStorage/IDatabase object first and then add it to IDatabase/DatabaseCatalog.
Expand Down

0 comments on commit a47b2c1

Please sign in to comment.