mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #51149 from ClickHouse/revert-48821-localfilefunction
Revert "Added ability to implicitly use file/hdfs/s3 table functions in clickhouse-local"
This commit is contained in:
commit
a09e6bbb8e
@ -8,9 +8,7 @@
|
|||||||
#include <Poco/Logger.h>
|
#include <Poco/Logger.h>
|
||||||
#include <Poco/NullChannel.h>
|
#include <Poco/NullChannel.h>
|
||||||
#include <Poco/SimpleFileChannel.h>
|
#include <Poco/SimpleFileChannel.h>
|
||||||
#include <Databases/DatabaseFilesystem.h>
|
|
||||||
#include <Databases/DatabaseMemory.h>
|
#include <Databases/DatabaseMemory.h>
|
||||||
#include <Databases/DatabasesOverlay.h>
|
|
||||||
#include <Storages/System/attachSystemTables.h>
|
#include <Storages/System/attachSystemTables.h>
|
||||||
#include <Storages/System/attachInformationSchemaTables.h>
|
#include <Storages/System/attachInformationSchemaTables.h>
|
||||||
#include <Interpreters/DatabaseCatalog.h>
|
#include <Interpreters/DatabaseCatalog.h>
|
||||||
@ -52,8 +50,6 @@
|
|||||||
#include <base/argsToConfig.h>
|
#include <base/argsToConfig.h>
|
||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
|
|
||||||
#include "config.h"
|
|
||||||
|
|
||||||
#if defined(FUZZING_MODE)
|
#if defined(FUZZING_MODE)
|
||||||
#include <Functions/getFuzzerData.h>
|
#include <Functions/getFuzzerData.h>
|
||||||
#endif
|
#endif
|
||||||
@ -174,13 +170,6 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str
|
|||||||
return system_database;
|
return system_database;
|
||||||
}
|
}
|
||||||
|
|
||||||
static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_)
|
|
||||||
{
|
|
||||||
auto databaseCombiner = std::make_shared<DatabasesOverlay>(name_, context_);
|
|
||||||
databaseCombiner->registerNextDatabase(std::make_shared<DatabaseFilesystem>(name_, "", context_));
|
|
||||||
databaseCombiner->registerNextDatabase(std::make_shared<DatabaseMemory>(name_, context_));
|
|
||||||
return databaseCombiner;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// If path is specified and not empty, will try to setup server environment and load existing metadata
|
/// If path is specified and not empty, will try to setup server environment and load existing metadata
|
||||||
void LocalServer::tryInitPath()
|
void LocalServer::tryInitPath()
|
||||||
@ -680,7 +669,7 @@ void LocalServer::processConfig()
|
|||||||
* if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons.
|
* if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons.
|
||||||
*/
|
*/
|
||||||
std::string default_database = config().getString("default_database", "_local");
|
std::string default_database = config().getString("default_database", "_local");
|
||||||
DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context));
|
DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared<DatabaseMemory>(default_database, global_context));
|
||||||
global_context->setCurrentDatabase(default_database);
|
global_context->setCurrentDatabase(default_database);
|
||||||
applyCmdOptions(global_context);
|
applyCmdOptions(global_context);
|
||||||
|
|
||||||
|
@ -3,7 +3,6 @@
|
|||||||
#include <filesystem>
|
#include <filesystem>
|
||||||
#include <Databases/DatabaseAtomic.h>
|
#include <Databases/DatabaseAtomic.h>
|
||||||
#include <Databases/DatabaseDictionary.h>
|
#include <Databases/DatabaseDictionary.h>
|
||||||
#include <Databases/DatabaseFilesystem.h>
|
|
||||||
#include <Databases/DatabaseLazy.h>
|
#include <Databases/DatabaseLazy.h>
|
||||||
#include <Databases/DatabaseMemory.h>
|
#include <Databases/DatabaseMemory.h>
|
||||||
#include <Databases/DatabaseOrdinary.h>
|
#include <Databases/DatabaseOrdinary.h>
|
||||||
@ -48,14 +47,6 @@
|
|||||||
#include <Databases/SQLite/DatabaseSQLite.h>
|
#include <Databases/SQLite/DatabaseSQLite.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_AWS_S3
|
|
||||||
#include <Databases/DatabaseS3.h>
|
|
||||||
#endif
|
|
||||||
|
|
||||||
#if USE_HDFS
|
|
||||||
#include <Databases/DatabaseHDFS.h>
|
|
||||||
#endif
|
|
||||||
|
|
||||||
namespace fs = std::filesystem;
|
namespace fs = std::filesystem;
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -140,13 +131,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
|||||||
|
|
||||||
static const std::unordered_set<std::string_view> database_engines{"Ordinary", "Atomic", "Memory",
|
static const std::unordered_set<std::string_view> database_engines{"Ordinary", "Atomic", "Memory",
|
||||||
"Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
"Dictionary", "Lazy", "Replicated", "MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
||||||
"PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"};
|
"PostgreSQL", "MaterializedPostgreSQL", "SQLite"};
|
||||||
|
|
||||||
if (!database_engines.contains(engine_name))
|
if (!database_engines.contains(engine_name))
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name);
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name);
|
||||||
|
|
||||||
static const std::unordered_set<std::string_view> engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
static const std::unordered_set<std::string_view> engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
||||||
"Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"};
|
"Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite"};
|
||||||
|
|
||||||
static const std::unordered_set<std::string_view> engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"};
|
static const std::unordered_set<std::string_view> engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"};
|
||||||
bool engine_may_have_arguments = engines_with_arguments.contains(engine_name);
|
bool engine_may_have_arguments = engines_with_arguments.contains(engine_name);
|
||||||
@ -441,63 +432,6 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String
|
|||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
else if (engine_name == "Filesystem")
|
|
||||||
{
|
|
||||||
const ASTFunction * engine = engine_define->engine;
|
|
||||||
|
|
||||||
/// If init_path is empty, then the current path will be used
|
|
||||||
std::string init_path;
|
|
||||||
|
|
||||||
if (engine->arguments && !engine->arguments->children.empty())
|
|
||||||
{
|
|
||||||
if (engine->arguments->children.size() != 1)
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filesystem database requires at most 1 argument: filesystem_path");
|
|
||||||
|
|
||||||
const auto & arguments = engine->arguments->children;
|
|
||||||
init_path = safeGetLiteralValue<String>(arguments[0], engine_name);
|
|
||||||
}
|
|
||||||
|
|
||||||
return std::make_shared<DatabaseFilesystem>(database_name, init_path, context);
|
|
||||||
}
|
|
||||||
|
|
||||||
#if USE_AWS_S3
|
|
||||||
else if (engine_name == "S3")
|
|
||||||
{
|
|
||||||
const ASTFunction * engine = engine_define->engine;
|
|
||||||
|
|
||||||
DatabaseS3::Configuration config;
|
|
||||||
|
|
||||||
if (engine->arguments && !engine->arguments->children.empty())
|
|
||||||
{
|
|
||||||
ASTs & engine_args = engine->arguments->children;
|
|
||||||
config = DatabaseS3::parseArguments(engine_args, context);
|
|
||||||
}
|
|
||||||
|
|
||||||
return std::make_shared<DatabaseS3>(database_name, config, context);
|
|
||||||
}
|
|
||||||
#endif
|
|
||||||
|
|
||||||
#if USE_HDFS
|
|
||||||
else if (engine_name == "HDFS")
|
|
||||||
{
|
|
||||||
const ASTFunction * engine = engine_define->engine;
|
|
||||||
|
|
||||||
/// If source_url is empty, then table name must contain full url
|
|
||||||
std::string source_url;
|
|
||||||
|
|
||||||
if (engine->arguments && !engine->arguments->children.empty())
|
|
||||||
{
|
|
||||||
if (engine->arguments->children.size() != 1)
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS database requires at most 1 argument: source_url");
|
|
||||||
|
|
||||||
const auto & arguments = engine->arguments->children;
|
|
||||||
source_url = safeGetLiteralValue<String>(arguments[0], engine_name);
|
|
||||||
}
|
|
||||||
|
|
||||||
return std::make_shared<DatabaseHDFS>(database_name, source_url, context);
|
|
||||||
}
|
|
||||||
#endif
|
|
||||||
|
|
||||||
throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", engine_name);
|
throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", engine_name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,247 +0,0 @@
|
|||||||
#include <Databases/DatabaseFilesystem.h>
|
|
||||||
|
|
||||||
#include <IO/Operators.h>
|
|
||||||
#include <IO/WriteBufferFromString.h>
|
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
|
||||||
#include <Parsers/parseQuery.h>
|
|
||||||
#include <Storages/IStorage.h>
|
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
|
||||||
#include <Common/filesystemHelpers.h>
|
|
||||||
|
|
||||||
#include <filesystem>
|
|
||||||
|
|
||||||
namespace fs = std::filesystem;
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
extern const int UNKNOWN_TABLE;
|
|
||||||
extern const int DATABASE_ACCESS_DENIED;
|
|
||||||
extern const int BAD_ARGUMENTS;
|
|
||||||
extern const int FILE_DOESNT_EXIST;
|
|
||||||
}
|
|
||||||
|
|
||||||
DatabaseFilesystem::DatabaseFilesystem(const String & name_, const String & path_, ContextPtr context_)
|
|
||||||
: IDatabase(name_), WithContext(context_->getGlobalContext()), path(path_), log(&Poco::Logger::get("DatabaseFileSystem(" + name_ + ")"))
|
|
||||||
{
|
|
||||||
bool is_local = context_->getApplicationType() == Context::ApplicationType::LOCAL;
|
|
||||||
fs::path user_files_path = is_local ? "" : fs::canonical(getContext()->getUserFilesPath());
|
|
||||||
|
|
||||||
if (fs::path(path).is_relative())
|
|
||||||
{
|
|
||||||
path = user_files_path / path;
|
|
||||||
}
|
|
||||||
else if (!is_local && !pathStartsWith(fs::path(path), user_files_path))
|
|
||||||
{
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
||||||
"Path must be inside user-files path: {}", user_files_path.string());
|
|
||||||
}
|
|
||||||
|
|
||||||
path = fs::absolute(path).lexically_normal();
|
|
||||||
if (!fs::exists(path))
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Path does not exist: {}", path);
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string DatabaseFilesystem::getTablePath(const std::string & table_name) const
|
|
||||||
{
|
|
||||||
fs::path table_path = fs::path(path) / table_name;
|
|
||||||
return table_path.lexically_normal().string();
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabaseFilesystem::addTable(const std::string & table_name, StoragePtr table_storage) const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
auto [_, inserted] = loaded_tables.emplace(table_name, table_storage);
|
|
||||||
if (!inserted)
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"Table with name `{}` already exists in database `{}` (engine {})",
|
|
||||||
table_name, getDatabaseName(), getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseFilesystem::checkTableFilePath(const std::string & table_path, ContextPtr context_, bool throw_on_error) const
|
|
||||||
{
|
|
||||||
/// If run in Local mode, no need for path checking.
|
|
||||||
bool check_path = context_->getApplicationType() != Context::ApplicationType::LOCAL;
|
|
||||||
const auto & user_files_path = context_->getUserFilesPath();
|
|
||||||
|
|
||||||
/// Check access for file before checking its existence.
|
|
||||||
if (check_path && !fileOrSymlinkPathStartsWith(table_path, user_files_path))
|
|
||||||
{
|
|
||||||
if (throw_on_error)
|
|
||||||
throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File is not inside {}", user_files_path);
|
|
||||||
else
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Check if the corresponding file exists.
|
|
||||||
if (!fs::exists(table_path))
|
|
||||||
{
|
|
||||||
if (throw_on_error)
|
|
||||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "File does not exist: {}", table_path);
|
|
||||||
else
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!fs::is_regular_file(table_path))
|
|
||||||
{
|
|
||||||
if (throw_on_error)
|
|
||||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST,
|
|
||||||
"File is directory, but expected a file: {}", table_path);
|
|
||||||
else
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseFilesystem::tryGetTableFromCache(const std::string & name) const
|
|
||||||
{
|
|
||||||
StoragePtr table = nullptr;
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
auto it = loaded_tables.find(name);
|
|
||||||
if (it != loaded_tables.end())
|
|
||||||
table = it->second;
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Invalidate cache if file no longer exists.
|
|
||||||
if (table && !fs::exists(getTablePath(name)))
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
loaded_tables.erase(name);
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
return table;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseFilesystem::isTableExist(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
if (tryGetTableFromCache(name))
|
|
||||||
return true;
|
|
||||||
|
|
||||||
return checkTableFilePath(getTablePath(name), context_, /* throw_on_error */false);
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
/// Check if table exists in loaded tables map.
|
|
||||||
if (auto table = tryGetTableFromCache(name))
|
|
||||||
return table;
|
|
||||||
|
|
||||||
auto table_path = getTablePath(name);
|
|
||||||
checkTableFilePath(table_path, context_, /* throw_on_error */true);
|
|
||||||
|
|
||||||
/// If the file exists, create a new table using TableFunctionFile and return it.
|
|
||||||
auto args = makeASTFunction("file", std::make_shared<ASTLiteral>(table_path));
|
|
||||||
|
|
||||||
auto table_function = TableFunctionFactory::instance().get(args, context_);
|
|
||||||
if (!table_function)
|
|
||||||
return nullptr;
|
|
||||||
|
|
||||||
/// TableFunctionFile throws exceptions, if table cannot be created.
|
|
||||||
auto table_storage = table_function->execute(args, context_, name);
|
|
||||||
if (table_storage)
|
|
||||||
addTable(name, table_storage);
|
|
||||||
|
|
||||||
return table_storage;
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseFilesystem::getTable(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
/// getTableImpl can throw exceptions, do not catch them to show correct error to user.
|
|
||||||
if (auto storage = getTableImpl(name, context_))
|
|
||||||
return storage;
|
|
||||||
|
|
||||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
|
|
||||||
backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name));
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseFilesystem::tryGetTable(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
return getTableImpl(name, context_);
|
|
||||||
}
|
|
||||||
catch (const Exception & e)
|
|
||||||
{
|
|
||||||
/// Ignore exceptions thrown by TableFunctionFile, which indicate that there is no table
|
|
||||||
/// see tests/02722_database_filesystem.sh for more details.
|
|
||||||
if (e.code() == ErrorCodes::BAD_ARGUMENTS
|
|
||||||
|| e.code() == ErrorCodes::DATABASE_ACCESS_DENIED
|
|
||||||
|| e.code() == ErrorCodes::FILE_DOESNT_EXIST)
|
|
||||||
{
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
throw;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseFilesystem::empty() const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
return loaded_tables.empty();
|
|
||||||
}
|
|
||||||
|
|
||||||
ASTPtr DatabaseFilesystem::getCreateDatabaseQuery() const
|
|
||||||
{
|
|
||||||
const auto & settings = getContext()->getSettingsRef();
|
|
||||||
const String query = fmt::format("CREATE DATABASE {} ENGINE = Filesystem('{}')", backQuoteIfNeed(getDatabaseName()), path);
|
|
||||||
|
|
||||||
ParserCreateQuery parser;
|
|
||||||
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth);
|
|
||||||
|
|
||||||
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
|
|
||||||
{
|
|
||||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
|
||||||
ast_create_query.set(ast_create_query.comment, std::make_shared<ASTLiteral>(database_comment));
|
|
||||||
}
|
|
||||||
|
|
||||||
return ast;
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabaseFilesystem::shutdown()
|
|
||||||
{
|
|
||||||
Tables tables_snapshot;
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
tables_snapshot = loaded_tables;
|
|
||||||
}
|
|
||||||
|
|
||||||
for (const auto & kv : tables_snapshot)
|
|
||||||
{
|
|
||||||
auto table_id = kv.second->getStorageID();
|
|
||||||
kv.second->flushAndShutdown();
|
|
||||||
}
|
|
||||||
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
loaded_tables.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns an empty vector because the database is read-only and no tables can be backed up
|
|
||||||
*/
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> DatabaseFilesystem::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const
|
|
||||||
{
|
|
||||||
return {};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Returns an empty iterator because the database does not have its own tables
|
|
||||||
* But only caches them for quick access
|
|
||||||
*/
|
|
||||||
DatabaseTablesIteratorPtr DatabaseFilesystem::getTablesIterator(ContextPtr, const FilterByNameFunction &) const
|
|
||||||
{
|
|
||||||
return std::make_unique<DatabaseTablesSnapshotIterator>(Tables{}, getDatabaseName());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,67 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <mutex>
|
|
||||||
#include <Databases/IDatabase.h>
|
|
||||||
#include <Parsers/IAST.h>
|
|
||||||
#include <Storages/IStorage_fwd.h>
|
|
||||||
#include <base/types.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
class Context;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* DatabaseFilesystem allows to interact with files stored on the local filesystem.
|
|
||||||
* Uses TableFunctionFile to implicitly load file when a user requests the table,
|
|
||||||
* and provides a read-only access to the data in the file.
|
|
||||||
* Tables are cached inside the database for quick access
|
|
||||||
*
|
|
||||||
* Used in clickhouse-local to access local files.
|
|
||||||
* For clickhouse-server requires allows to access file only from user_files directory.
|
|
||||||
*/
|
|
||||||
class DatabaseFilesystem : public IDatabase, protected WithContext
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
DatabaseFilesystem(const String & name, const String & path, ContextPtr context);
|
|
||||||
|
|
||||||
String getEngineName() const override { return "Filesystem"; }
|
|
||||||
|
|
||||||
bool isTableExist(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
StoragePtr getTable(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
StoragePtr tryGetTable(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
bool shouldBeEmptyOnDetach() const override { return false; } /// Contains only temporary tables.
|
|
||||||
|
|
||||||
bool empty() const override;
|
|
||||||
|
|
||||||
bool isReadOnly() const override { return true; }
|
|
||||||
|
|
||||||
ASTPtr getCreateDatabaseQuery() const override;
|
|
||||||
|
|
||||||
void shutdown() override;
|
|
||||||
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override;
|
|
||||||
|
|
||||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override;
|
|
||||||
|
|
||||||
protected:
|
|
||||||
StoragePtr getTableImpl(const String & name, ContextPtr context) const;
|
|
||||||
|
|
||||||
StoragePtr tryGetTableFromCache(const std::string & name) const;
|
|
||||||
|
|
||||||
std::string getTablePath(const std::string & table_name) const;
|
|
||||||
|
|
||||||
void addTable(const std::string & table_name, StoragePtr table_storage) const;
|
|
||||||
|
|
||||||
bool checkTableFilePath(const std::string & table_path, ContextPtr context_, bool throw_on_error) const;
|
|
||||||
|
|
||||||
private:
|
|
||||||
String path;
|
|
||||||
mutable Tables loaded_tables TSA_GUARDED_BY(mutex);
|
|
||||||
Poco::Logger * log;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -1,234 +0,0 @@
|
|||||||
#include "config.h"
|
|
||||||
|
|
||||||
#if USE_HDFS
|
|
||||||
|
|
||||||
#include <Databases/DatabaseHDFS.h>
|
|
||||||
|
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
|
||||||
#include <Parsers/parseQuery.h>
|
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
|
||||||
#include <Storages/HDFS/HDFSCommon.h>
|
|
||||||
#include <Storages/IStorage.h>
|
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
|
||||||
|
|
||||||
#include <Poco/URI.h>
|
|
||||||
#include <re2/re2.h>
|
|
||||||
|
|
||||||
#include <filesystem>
|
|
||||||
|
|
||||||
namespace fs = std::filesystem;
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
extern const int UNKNOWN_TABLE;
|
|
||||||
extern const int BAD_ARGUMENTS;
|
|
||||||
extern const int FILE_DOESNT_EXIST;
|
|
||||||
extern const int UNACCEPTABLE_URL;
|
|
||||||
extern const int ACCESS_DENIED;
|
|
||||||
extern const int DATABASE_ACCESS_DENIED;
|
|
||||||
extern const int HDFS_ERROR;
|
|
||||||
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
|
||||||
}
|
|
||||||
|
|
||||||
static constexpr std::string_view HDFS_HOST_REGEXP = "^hdfs://[^/]*";
|
|
||||||
|
|
||||||
|
|
||||||
DatabaseHDFS::DatabaseHDFS(const String & name_, const String & source_url, ContextPtr context_)
|
|
||||||
: IDatabase(name_)
|
|
||||||
, WithContext(context_->getGlobalContext())
|
|
||||||
, source(source_url)
|
|
||||||
, log(&Poco::Logger::get("DatabaseHDFS(" + name_ + ")"))
|
|
||||||
{
|
|
||||||
if (!source.empty())
|
|
||||||
{
|
|
||||||
if (!re2::RE2::FullMatch(source, std::string(HDFS_HOST_REGEXP)))
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs host: {}. "
|
|
||||||
"It should have structure 'hdfs://<host_name>:<port>'", source);
|
|
||||||
|
|
||||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(source));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabaseHDFS::addTable(const std::string & table_name, StoragePtr table_storage) const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
auto [_, inserted] = loaded_tables.emplace(table_name, table_storage);
|
|
||||||
if (!inserted)
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"Table with name `{}` already exists in database `{}` (engine {})",
|
|
||||||
table_name, getDatabaseName(), getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string DatabaseHDFS::getTablePath(const std::string & table_name) const
|
|
||||||
{
|
|
||||||
if (table_name.starts_with("hdfs://"))
|
|
||||||
return table_name;
|
|
||||||
|
|
||||||
if (source.empty())
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}. "
|
|
||||||
"It should have structure 'hdfs://<host_name>:<port>/path'", table_name);
|
|
||||||
|
|
||||||
return fs::path(source) / table_name;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseHDFS::checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
checkHDFSURL(url);
|
|
||||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(Poco::URI(url));
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
if (throw_on_error)
|
|
||||||
throw;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseHDFS::isTableExist(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
if (loaded_tables.find(name) != loaded_tables.end())
|
|
||||||
return true;
|
|
||||||
|
|
||||||
return checkUrl(name, context_, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseHDFS::getTableImpl(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
/// Check if the table exists in the loaded tables map.
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
auto it = loaded_tables.find(name);
|
|
||||||
if (it != loaded_tables.end())
|
|
||||||
return it->second;
|
|
||||||
}
|
|
||||||
|
|
||||||
auto url = getTablePath(name);
|
|
||||||
|
|
||||||
checkUrl(url, context_, true);
|
|
||||||
|
|
||||||
auto args = makeASTFunction("hdfs", std::make_shared<ASTLiteral>(url));
|
|
||||||
|
|
||||||
auto table_function = TableFunctionFactory::instance().get(args, context_);
|
|
||||||
if (!table_function)
|
|
||||||
return nullptr;
|
|
||||||
|
|
||||||
/// TableFunctionHDFS throws exceptions, if table cannot be created.
|
|
||||||
auto table_storage = table_function->execute(args, context_, name);
|
|
||||||
if (table_storage)
|
|
||||||
addTable(name, table_storage);
|
|
||||||
|
|
||||||
return table_storage;
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseHDFS::getTable(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
/// Rethrow all exceptions from TableFunctionHDFS to show correct error to user.
|
|
||||||
if (auto storage = getTableImpl(name, context_))
|
|
||||||
return storage;
|
|
||||||
|
|
||||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
|
|
||||||
backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name));
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseHDFS::tryGetTable(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
return getTableImpl(name, context_);
|
|
||||||
}
|
|
||||||
catch (const Exception & e)
|
|
||||||
{
|
|
||||||
// Ignore exceptions thrown by TableFunctionHDFS, which indicate that there is no table
|
|
||||||
if (e.code() == ErrorCodes::BAD_ARGUMENTS
|
|
||||||
|| e.code() == ErrorCodes::ACCESS_DENIED
|
|
||||||
|| e.code() == ErrorCodes::DATABASE_ACCESS_DENIED
|
|
||||||
|| e.code() == ErrorCodes::FILE_DOESNT_EXIST
|
|
||||||
|| e.code() == ErrorCodes::UNACCEPTABLE_URL
|
|
||||||
|| e.code() == ErrorCodes::HDFS_ERROR
|
|
||||||
|| e.code() == ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE)
|
|
||||||
{
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
throw;
|
|
||||||
}
|
|
||||||
catch (const Poco::URISyntaxException &)
|
|
||||||
{
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseHDFS::empty() const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
return loaded_tables.empty();
|
|
||||||
}
|
|
||||||
|
|
||||||
ASTPtr DatabaseHDFS::getCreateDatabaseQuery() const
|
|
||||||
{
|
|
||||||
const auto & settings = getContext()->getSettingsRef();
|
|
||||||
ParserCreateQuery parser;
|
|
||||||
|
|
||||||
const String query = fmt::format("CREATE DATABASE {} ENGINE = HDFS('{}')", backQuoteIfNeed(getDatabaseName()), source);
|
|
||||||
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth);
|
|
||||||
|
|
||||||
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
|
|
||||||
{
|
|
||||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
|
||||||
ast_create_query.set(ast_create_query.comment, std::make_shared<ASTLiteral>(database_comment));
|
|
||||||
}
|
|
||||||
|
|
||||||
return ast;
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabaseHDFS::shutdown()
|
|
||||||
{
|
|
||||||
Tables tables_snapshot;
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
tables_snapshot = loaded_tables;
|
|
||||||
}
|
|
||||||
|
|
||||||
for (const auto & kv : tables_snapshot)
|
|
||||||
{
|
|
||||||
auto table_id = kv.second->getStorageID();
|
|
||||||
kv.second->flushAndShutdown();
|
|
||||||
}
|
|
||||||
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
loaded_tables.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns an empty vector because the database is read-only and no tables can be backed up
|
|
||||||
*/
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> DatabaseHDFS::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const
|
|
||||||
{
|
|
||||||
return {};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Returns an empty iterator because the database does not have its own tables
|
|
||||||
* But only caches them for quick access
|
|
||||||
*/
|
|
||||||
DatabaseTablesIteratorPtr DatabaseHDFS::getTablesIterator(ContextPtr, const FilterByNameFunction &) const
|
|
||||||
{
|
|
||||||
return std::make_unique<DatabaseTablesSnapshotIterator>(Tables{}, getDatabaseName());
|
|
||||||
}
|
|
||||||
|
|
||||||
} // DB
|
|
||||||
|
|
||||||
#endif
|
|
@ -1,68 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include "config.h"
|
|
||||||
|
|
||||||
#if USE_HDFS
|
|
||||||
|
|
||||||
#include <mutex>
|
|
||||||
#include <Databases/IDatabase.h>
|
|
||||||
#include <Parsers/IAST.h>
|
|
||||||
#include <Storages/IStorage_fwd.h>
|
|
||||||
#include <base/types.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
class Context;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* DatabaseHDFS allows to interact with files stored on the file system.
|
|
||||||
* Uses TableFunctionHDFS to implicitly load file when a user requests the table,
|
|
||||||
* and provides read-only access to the data in the file.
|
|
||||||
* Tables are cached inside the database for quick access.
|
|
||||||
*/
|
|
||||||
class DatabaseHDFS : public IDatabase, protected WithContext
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
DatabaseHDFS(const String & name, const String & source_url, ContextPtr context);
|
|
||||||
|
|
||||||
String getEngineName() const override { return "S3"; }
|
|
||||||
|
|
||||||
bool isTableExist(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
StoragePtr getTable(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
StoragePtr tryGetTable(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
bool shouldBeEmptyOnDetach() const override { return false; } /// Contains only temporary tables.
|
|
||||||
|
|
||||||
bool empty() const override;
|
|
||||||
|
|
||||||
bool isReadOnly() const override { return true; }
|
|
||||||
|
|
||||||
ASTPtr getCreateDatabaseQuery() const override;
|
|
||||||
|
|
||||||
void shutdown() override;
|
|
||||||
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override;
|
|
||||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override;
|
|
||||||
|
|
||||||
protected:
|
|
||||||
StoragePtr getTableImpl(const String & name, ContextPtr context) const;
|
|
||||||
|
|
||||||
void addTable(const std::string & table_name, StoragePtr table_storage) const;
|
|
||||||
|
|
||||||
bool checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const;
|
|
||||||
|
|
||||||
std::string getTablePath(const std::string & table_name) const;
|
|
||||||
|
|
||||||
private:
|
|
||||||
const String source;
|
|
||||||
|
|
||||||
mutable Tables loaded_tables TSA_GUARDED_BY(mutex);
|
|
||||||
Poco::Logger * log;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
#endif
|
|
@ -1,312 +0,0 @@
|
|||||||
#include "config.h"
|
|
||||||
|
|
||||||
#if USE_AWS_S3
|
|
||||||
|
|
||||||
#include <Databases/DatabaseS3.h>
|
|
||||||
|
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Interpreters/evaluateConstantExpression.h>
|
|
||||||
#include <IO/S3/URI.h>
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
|
||||||
#include <Parsers/ASTFunction.h>
|
|
||||||
#include <Parsers/ASTLiteral.h>
|
|
||||||
#include <Parsers/parseQuery.h>
|
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
|
||||||
#include <Storages/checkAndGetLiteralArgument.h>
|
|
||||||
#include <Storages/IStorage.h>
|
|
||||||
#include <Storages/NamedCollectionsHelpers.h>
|
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
|
||||||
|
|
||||||
#include <boost/algorithm/string.hpp>
|
|
||||||
#include <filesystem>
|
|
||||||
|
|
||||||
namespace fs = std::filesystem;
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
static const std::unordered_set<std::string_view> optional_configuration_keys = {
|
|
||||||
"url",
|
|
||||||
"access_key_id",
|
|
||||||
"secret_access_key",
|
|
||||||
"no_sign_request"
|
|
||||||
};
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
extern const int UNKNOWN_TABLE;
|
|
||||||
extern const int BAD_ARGUMENTS;
|
|
||||||
extern const int FILE_DOESNT_EXIST;
|
|
||||||
extern const int UNACCEPTABLE_URL;
|
|
||||||
extern const int S3_ERROR;
|
|
||||||
|
|
||||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
||||||
}
|
|
||||||
|
|
||||||
DatabaseS3::DatabaseS3(const String & name_, const Configuration& config_, ContextPtr context_)
|
|
||||||
: IDatabase(name_)
|
|
||||||
, WithContext(context_->getGlobalContext())
|
|
||||||
, config(config_)
|
|
||||||
, log(&Poco::Logger::get("DatabaseS3(" + name_ + ")"))
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabaseS3::addTable(const std::string & table_name, StoragePtr table_storage) const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
auto [_, inserted] = loaded_tables.emplace(table_name, table_storage);
|
|
||||||
if (!inserted)
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"Table with name `{}` already exists in database `{}` (engine {})",
|
|
||||||
table_name, getDatabaseName(), getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
std::string DatabaseS3::getFullUrl(const std::string & name) const
|
|
||||||
{
|
|
||||||
if (!config.url_prefix.empty())
|
|
||||||
return fs::path(config.url_prefix) / name;
|
|
||||||
|
|
||||||
return name;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseS3::checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
S3::URI uri(url);
|
|
||||||
context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri.uri);
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
if (throw_on_error)
|
|
||||||
throw;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseS3::isTableExist(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
if (loaded_tables.find(name) != loaded_tables.end())
|
|
||||||
return true;
|
|
||||||
|
|
||||||
return checkUrl(getFullUrl(name), context_, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseS3::getTableImpl(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
/// Check if the table exists in the loaded tables map.
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
auto it = loaded_tables.find(name);
|
|
||||||
if (it != loaded_tables.end())
|
|
||||||
return it->second;
|
|
||||||
}
|
|
||||||
|
|
||||||
auto url = getFullUrl(name);
|
|
||||||
checkUrl(url, context_, /* throw_on_error */true);
|
|
||||||
|
|
||||||
auto function = std::make_shared<ASTFunction>();
|
|
||||||
function->name = "s3";
|
|
||||||
function->arguments = std::make_shared<ASTExpressionList>();
|
|
||||||
function->children.push_back(function->arguments);
|
|
||||||
|
|
||||||
function->arguments->children.push_back(std::make_shared<ASTLiteral>(url));
|
|
||||||
if (config.no_sign_request)
|
|
||||||
{
|
|
||||||
function->arguments->children.push_back(std::make_shared<ASTLiteral>("NOSIGN"));
|
|
||||||
}
|
|
||||||
else if (config.access_key_id.has_value() && config.secret_access_key.has_value())
|
|
||||||
{
|
|
||||||
function->arguments->children.push_back(std::make_shared<ASTLiteral>(config.access_key_id.value()));
|
|
||||||
function->arguments->children.push_back(std::make_shared<ASTLiteral>(config.secret_access_key.value()));
|
|
||||||
}
|
|
||||||
|
|
||||||
auto table_function = TableFunctionFactory::instance().get(function, context_);
|
|
||||||
if (!table_function)
|
|
||||||
return nullptr;
|
|
||||||
|
|
||||||
/// TableFunctionS3 throws exceptions, if table cannot be created.
|
|
||||||
auto table_storage = table_function->execute(function, context_, name);
|
|
||||||
if (table_storage)
|
|
||||||
addTable(name, table_storage);
|
|
||||||
|
|
||||||
return table_storage;
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseS3::getTable(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
/// Rethrow all exceptions from TableFunctionS3 to show correct error to user.
|
|
||||||
if (auto storage = getTableImpl(name, context_))
|
|
||||||
return storage;
|
|
||||||
|
|
||||||
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
|
|
||||||
backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name));
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabaseS3::tryGetTable(const String & name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
return getTableImpl(name, context_);
|
|
||||||
}
|
|
||||||
catch (const Exception & e)
|
|
||||||
{
|
|
||||||
/// Ignore exceptions thrown by TableFunctionS3, which indicate that there is no table.
|
|
||||||
if (e.code() == ErrorCodes::BAD_ARGUMENTS
|
|
||||||
|| e.code() == ErrorCodes::S3_ERROR
|
|
||||||
|| e.code() == ErrorCodes::FILE_DOESNT_EXIST
|
|
||||||
|| e.code() == ErrorCodes::UNACCEPTABLE_URL)
|
|
||||||
{
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
throw;
|
|
||||||
}
|
|
||||||
catch (const Poco::URISyntaxException &)
|
|
||||||
{
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabaseS3::empty() const
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
return loaded_tables.empty();
|
|
||||||
}
|
|
||||||
|
|
||||||
ASTPtr DatabaseS3::getCreateDatabaseQuery() const
|
|
||||||
{
|
|
||||||
const auto & settings = getContext()->getSettingsRef();
|
|
||||||
ParserCreateQuery parser;
|
|
||||||
|
|
||||||
std::string creation_args;
|
|
||||||
creation_args += fmt::format("'{}'", config.url_prefix);
|
|
||||||
if (config.no_sign_request)
|
|
||||||
creation_args += ", 'NOSIGN'";
|
|
||||||
else if (config.access_key_id.has_value() && config.secret_access_key.has_value())
|
|
||||||
creation_args += fmt::format(", '{}', '{}'", config.access_key_id.value(), config.secret_access_key.value());
|
|
||||||
|
|
||||||
const String query = fmt::format("CREATE DATABASE {} ENGINE = S3({})", backQuoteIfNeed(getDatabaseName()), creation_args);
|
|
||||||
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth);
|
|
||||||
|
|
||||||
if (const auto database_comment = getDatabaseComment(); !database_comment.empty())
|
|
||||||
{
|
|
||||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
|
||||||
ast_create_query.set(ast_create_query.comment, std::make_shared<ASTLiteral>(database_comment));
|
|
||||||
}
|
|
||||||
|
|
||||||
return ast;
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabaseS3::shutdown()
|
|
||||||
{
|
|
||||||
Tables tables_snapshot;
|
|
||||||
{
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
tables_snapshot = loaded_tables;
|
|
||||||
}
|
|
||||||
|
|
||||||
for (const auto & kv : tables_snapshot)
|
|
||||||
{
|
|
||||||
auto table_id = kv.second->getStorageID();
|
|
||||||
kv.second->flushAndShutdown();
|
|
||||||
}
|
|
||||||
|
|
||||||
std::lock_guard lock(mutex);
|
|
||||||
loaded_tables.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
DatabaseS3::Configuration DatabaseS3::parseArguments(ASTs engine_args, ContextPtr context_)
|
|
||||||
{
|
|
||||||
Configuration result;
|
|
||||||
|
|
||||||
if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, context_))
|
|
||||||
{
|
|
||||||
auto & collection = *named_collection;
|
|
||||||
|
|
||||||
validateNamedCollection(collection, {}, optional_configuration_keys);
|
|
||||||
|
|
||||||
result.url_prefix = collection.getOrDefault<String>("url", "");
|
|
||||||
result.no_sign_request = collection.getOrDefault<bool>("no_sign_request", false);
|
|
||||||
|
|
||||||
auto key_id = collection.getOrDefault<String>("access_key_id", "");
|
|
||||||
auto secret_key = collection.getOrDefault<String>("secret_access_key", "");
|
|
||||||
|
|
||||||
if (!key_id.empty())
|
|
||||||
result.access_key_id = key_id;
|
|
||||||
|
|
||||||
if (!secret_key.empty())
|
|
||||||
result.secret_access_key = secret_key;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
const std::string supported_signature =
|
|
||||||
" - S3()\n"
|
|
||||||
" - S3('url')\n"
|
|
||||||
" - S3('url', 'NOSIGN')\n"
|
|
||||||
" - S3('url', 'access_key_id', 'secret_access_key')\n";
|
|
||||||
const auto error_message =
|
|
||||||
fmt::format("Engine DatabaseS3 must have the following arguments signature\n{}", supported_signature);
|
|
||||||
|
|
||||||
for (auto & arg : engine_args)
|
|
||||||
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context_);
|
|
||||||
|
|
||||||
if (engine_args.size() > 3)
|
|
||||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message.c_str());
|
|
||||||
|
|
||||||
if (engine_args.empty())
|
|
||||||
return result;
|
|
||||||
|
|
||||||
result.url_prefix = checkAndGetLiteralArgument<String>(engine_args[0], "url");
|
|
||||||
|
|
||||||
// url, NOSIGN
|
|
||||||
if (engine_args.size() == 2)
|
|
||||||
{
|
|
||||||
auto second_arg = checkAndGetLiteralArgument<String>(engine_args[1], "NOSIGN");
|
|
||||||
if (boost::iequals(second_arg, "NOSIGN"))
|
|
||||||
result.no_sign_request = true;
|
|
||||||
else
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message.c_str());
|
|
||||||
}
|
|
||||||
|
|
||||||
// url, access_key_id, secret_access_key
|
|
||||||
if (engine_args.size() == 3)
|
|
||||||
{
|
|
||||||
auto key_id = checkAndGetLiteralArgument<String>(engine_args[1], "access_key_id");
|
|
||||||
auto secret_key = checkAndGetLiteralArgument<String>(engine_args[2], "secret_access_key");
|
|
||||||
|
|
||||||
if (key_id.empty() || secret_key.empty() || boost::iequals(key_id, "NOSIGN"))
|
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message.c_str());
|
|
||||||
|
|
||||||
result.access_key_id = key_id;
|
|
||||||
result.secret_access_key = secret_key;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns an empty vector because the database is read-only and no tables can be backed up
|
|
||||||
*/
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> DatabaseS3::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const
|
|
||||||
{
|
|
||||||
return {};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Returns an empty iterator because the database does not have its own tables
|
|
||||||
* But only caches them for quick access
|
|
||||||
*/
|
|
||||||
DatabaseTablesIteratorPtr DatabaseS3::getTablesIterator(ContextPtr, const FilterByNameFunction &) const
|
|
||||||
{
|
|
||||||
return std::make_unique<DatabaseTablesSnapshotIterator>(Tables{}, getDatabaseName());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
#endif
|
|
@ -1,81 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include "config.h"
|
|
||||||
|
|
||||||
#if USE_AWS_S3
|
|
||||||
|
|
||||||
#include <mutex>
|
|
||||||
#include <Databases/IDatabase.h>
|
|
||||||
#include <Parsers/IAST.h>
|
|
||||||
#include <Storages/IStorage_fwd.h>
|
|
||||||
#include <base/types.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
class Context;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* DatabaseS3 provides access to data stored in S3.
|
|
||||||
* Uses TableFunctionS3 to implicitly load file when a user requests the table,
|
|
||||||
* and provides read-only access to the data in the file.
|
|
||||||
* Tables are cached inside the database for quick access.
|
|
||||||
*/
|
|
||||||
class DatabaseS3 : public IDatabase, protected WithContext
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
struct Configuration
|
|
||||||
{
|
|
||||||
std::string url_prefix;
|
|
||||||
|
|
||||||
bool no_sign_request = false;
|
|
||||||
|
|
||||||
std::optional<std::string> access_key_id;
|
|
||||||
std::optional<std::string> secret_access_key;
|
|
||||||
};
|
|
||||||
|
|
||||||
DatabaseS3(const String & name, const Configuration& config, ContextPtr context);
|
|
||||||
|
|
||||||
String getEngineName() const override { return "S3"; }
|
|
||||||
|
|
||||||
bool isTableExist(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
StoragePtr getTable(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
StoragePtr tryGetTable(const String & name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
// Contains only temporary tables
|
|
||||||
bool shouldBeEmptyOnDetach() const override { return false; }
|
|
||||||
|
|
||||||
bool empty() const override;
|
|
||||||
|
|
||||||
bool isReadOnly() const override { return true; }
|
|
||||||
|
|
||||||
ASTPtr getCreateDatabaseQuery() const override;
|
|
||||||
|
|
||||||
void shutdown() override;
|
|
||||||
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const override;
|
|
||||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr, const FilterByNameFunction &) const override;
|
|
||||||
|
|
||||||
static Configuration parseArguments(ASTs engine_args, ContextPtr context);
|
|
||||||
|
|
||||||
protected:
|
|
||||||
StoragePtr getTableImpl(const String & name, ContextPtr context) const;
|
|
||||||
|
|
||||||
void addTable(const std::string & table_name, StoragePtr table_storage) const;
|
|
||||||
|
|
||||||
bool checkUrl(const std::string & url, ContextPtr context_, bool throw_on_error) const;
|
|
||||||
|
|
||||||
std::string getFullUrl(const std::string & name) const;
|
|
||||||
|
|
||||||
private:
|
|
||||||
const Configuration config;
|
|
||||||
|
|
||||||
mutable Tables loaded_tables TSA_GUARDED_BY(mutex);
|
|
||||||
Poco::Logger * log;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
#endif
|
|
@ -1,266 +0,0 @@
|
|||||||
#include <Databases/DatabasesOverlay.h>
|
|
||||||
|
|
||||||
#include <Common/typeid_cast.h>
|
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Interpreters/InterpreterCreateQuery.h>
|
|
||||||
#include <Parsers/ASTCreateQuery.h>
|
|
||||||
|
|
||||||
#include <Storages/IStorage_fwd.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
namespace ErrorCodes
|
|
||||||
{
|
|
||||||
extern const int LOGICAL_ERROR;
|
|
||||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
|
||||||
}
|
|
||||||
|
|
||||||
DatabasesOverlay::DatabasesOverlay(const String & name_, ContextPtr context_)
|
|
||||||
: IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get("DatabaseOverlay(" + name_ + ")"))
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
DatabasesOverlay & DatabasesOverlay::registerNextDatabase(DatabasePtr database)
|
|
||||||
{
|
|
||||||
databases.push_back(std::move(database));
|
|
||||||
return *this;
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabasesOverlay::isTableExist(const String & table_name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
if (db->isTableExist(table_name, context_))
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabasesOverlay::tryGetTable(const String & table_name, ContextPtr context_) const
|
|
||||||
{
|
|
||||||
StoragePtr result = nullptr;
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
result = db->tryGetTable(table_name, context_);
|
|
||||||
if (result)
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabasesOverlay::createTable(ContextPtr context_, const String & table_name, const StoragePtr & table, const ASTPtr & query)
|
|
||||||
{
|
|
||||||
for (auto & db : databases)
|
|
||||||
{
|
|
||||||
if (!db->isReadOnly())
|
|
||||||
{
|
|
||||||
db->createTable(context_, table_name, table, query);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"There is no databases for CREATE TABLE `{}` query in database `{}` (engine {})",
|
|
||||||
table_name,
|
|
||||||
getDatabaseName(),
|
|
||||||
getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabasesOverlay::dropTable(ContextPtr context_, const String & table_name, bool sync)
|
|
||||||
{
|
|
||||||
for (auto & db : databases)
|
|
||||||
{
|
|
||||||
if (db->isTableExist(table_name, context_))
|
|
||||||
{
|
|
||||||
db->dropTable(context_, table_name, sync);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"There is no databases for DROP TABLE `{}` query in database `{}` (engine {})",
|
|
||||||
table_name,
|
|
||||||
getDatabaseName(),
|
|
||||||
getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabasesOverlay::attachTable(
|
|
||||||
ContextPtr context_, const String & table_name, const StoragePtr & table, const String & relative_table_path)
|
|
||||||
{
|
|
||||||
for (auto & db : databases)
|
|
||||||
{
|
|
||||||
try
|
|
||||||
{
|
|
||||||
db->attachTable(context_, table_name, table, relative_table_path);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"There is no databases for ATTACH TABLE `{}` query in database `{}` (engine {})",
|
|
||||||
table_name,
|
|
||||||
getDatabaseName(),
|
|
||||||
getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & table_name)
|
|
||||||
{
|
|
||||||
StoragePtr result = nullptr;
|
|
||||||
for (auto & db : databases)
|
|
||||||
{
|
|
||||||
if (db->isTableExist(table_name, context_))
|
|
||||||
return db->detachTable(context_, table_name);
|
|
||||||
}
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"There is no databases for DETACH TABLE `{}` query in database `{}` (engine {})",
|
|
||||||
table_name,
|
|
||||||
getDatabaseName(),
|
|
||||||
getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
ASTPtr DatabasesOverlay::getCreateTableQueryImpl(const String & name, ContextPtr context_, bool throw_on_error) const
|
|
||||||
{
|
|
||||||
ASTPtr result = nullptr;
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
result = db->tryGetCreateTableQuery(name, context_);
|
|
||||||
if (result)
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
if (!result && throw_on_error)
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY,
|
|
||||||
"There is no metadata of table `{}` in database `{}` (engine {})",
|
|
||||||
name,
|
|
||||||
getDatabaseName(),
|
|
||||||
getEngineName());
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* DatabaseOverlay cannot be constructed by "CREATE DATABASE" query, as it is not a traditional ClickHouse database
|
|
||||||
* To use DatabaseOverlay, it must be constructed programmatically in code
|
|
||||||
*/
|
|
||||||
ASTPtr DatabasesOverlay::getCreateDatabaseQuery() const
|
|
||||||
{
|
|
||||||
return std::make_shared<ASTCreateQuery>();
|
|
||||||
}
|
|
||||||
|
|
||||||
String DatabasesOverlay::getTableDataPath(const String & table_name) const
|
|
||||||
{
|
|
||||||
String result;
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
result = db->getTableDataPath(table_name);
|
|
||||||
if (!result.empty())
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
String DatabasesOverlay::getTableDataPath(const ASTCreateQuery & query) const
|
|
||||||
{
|
|
||||||
String result;
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
result = db->getTableDataPath(query);
|
|
||||||
if (!result.empty())
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
UUID DatabasesOverlay::tryGetTableUUID(const String & table_name) const
|
|
||||||
{
|
|
||||||
UUID result = UUIDHelpers::Nil;
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
result = db->tryGetTableUUID(table_name);
|
|
||||||
if (result != UUIDHelpers::Nil)
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabasesOverlay::drop(ContextPtr context_)
|
|
||||||
{
|
|
||||||
for (auto & db : databases)
|
|
||||||
db->drop(context_);
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabasesOverlay::alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata)
|
|
||||||
{
|
|
||||||
for (auto & db : databases)
|
|
||||||
{
|
|
||||||
if (!db->isReadOnly() && db->isTableExist(table_id.table_name, local_context))
|
|
||||||
{
|
|
||||||
db->alterTable(local_context, table_id, metadata);
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
throw Exception(
|
|
||||||
ErrorCodes::LOGICAL_ERROR,
|
|
||||||
"There is no databases for ALTER TABLE `{}` query in database `{}` (engine {})",
|
|
||||||
table_id.table_name,
|
|
||||||
getDatabaseName(),
|
|
||||||
getEngineName());
|
|
||||||
}
|
|
||||||
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>>
|
|
||||||
DatabasesOverlay::getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const
|
|
||||||
{
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> result;
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
auto db_backup = db->getTablesForBackup(filter, local_context);
|
|
||||||
result.insert(result.end(), std::make_move_iterator(db_backup.begin()), std::make_move_iterator(db_backup.end()));
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabasesOverlay::createTableRestoredFromBackup(
|
|
||||||
const ASTPtr & create_table_query,
|
|
||||||
ContextMutablePtr local_context,
|
|
||||||
std::shared_ptr<IRestoreCoordination> /*restore_coordination*/,
|
|
||||||
UInt64 /*timeout_ms*/)
|
|
||||||
{
|
|
||||||
/// Creates a tables by executing a "CREATE TABLE" query.
|
|
||||||
InterpreterCreateQuery interpreter{create_table_query, local_context};
|
|
||||||
interpreter.setInternal(true);
|
|
||||||
interpreter.execute();
|
|
||||||
}
|
|
||||||
|
|
||||||
bool DatabasesOverlay::empty() const
|
|
||||||
{
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
if (!db->empty())
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
void DatabasesOverlay::shutdown()
|
|
||||||
{
|
|
||||||
for (auto & db : databases)
|
|
||||||
db->shutdown();
|
|
||||||
}
|
|
||||||
|
|
||||||
DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context_, const FilterByNameFunction & filter_by_table_name) const
|
|
||||||
{
|
|
||||||
Tables tables;
|
|
||||||
for (const auto & db : databases)
|
|
||||||
{
|
|
||||||
for (auto table_it = db->getTablesIterator(context_, filter_by_table_name); table_it->isValid(); table_it->next())
|
|
||||||
tables.insert({table_it->name(), table_it->table()});
|
|
||||||
}
|
|
||||||
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(tables), getDatabaseName());
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,66 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <Storages/IStorage_fwd.h>
|
|
||||||
#include <Databases/IDatabase.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Implements the IDatabase interface and combines multiple other databases
|
|
||||||
* Searches for tables in each database in order until found, and delegates operations to the appropriate database
|
|
||||||
* Useful for combining databases
|
|
||||||
*
|
|
||||||
* Used in clickhouse-local to combine DatabaseFileSystem and DatabaseMemory
|
|
||||||
*/
|
|
||||||
class DatabasesOverlay : public IDatabase, protected WithContext
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
DatabasesOverlay(const String & name_, ContextPtr context_);
|
|
||||||
|
|
||||||
/// Not thread-safe. Use only as factory to initialize database
|
|
||||||
DatabasesOverlay & registerNextDatabase(DatabasePtr database);
|
|
||||||
|
|
||||||
String getEngineName() const override { return "Overlay"; }
|
|
||||||
|
|
||||||
public:
|
|
||||||
bool isTableExist(const String & table_name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
StoragePtr tryGetTable(const String & table_name, ContextPtr context) const override;
|
|
||||||
|
|
||||||
void createTable(ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override;
|
|
||||||
|
|
||||||
void dropTable(ContextPtr context, const String & table_name, bool sync) override;
|
|
||||||
|
|
||||||
void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override;
|
|
||||||
|
|
||||||
StoragePtr detachTable(ContextPtr context, const String & table_name) override;
|
|
||||||
|
|
||||||
ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override;
|
|
||||||
ASTPtr getCreateDatabaseQuery() const override;
|
|
||||||
|
|
||||||
String getTableDataPath(const String & table_name) const override;
|
|
||||||
String getTableDataPath(const ASTCreateQuery & query) const override;
|
|
||||||
|
|
||||||
UUID tryGetTableUUID(const String & table_name) const override;
|
|
||||||
|
|
||||||
void drop(ContextPtr context) override;
|
|
||||||
|
|
||||||
void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override;
|
|
||||||
|
|
||||||
std::vector<std::pair<ASTPtr, StoragePtr>> getTablesForBackup(const FilterByNameFunction & filter, const ContextPtr & local_context) const override;
|
|
||||||
|
|
||||||
void createTableRestoredFromBackup(const ASTPtr & create_table_query, ContextMutablePtr local_context, std::shared_ptr<IRestoreCoordination> restore_coordination, UInt64 timeout_ms) override;
|
|
||||||
|
|
||||||
DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override;
|
|
||||||
|
|
||||||
bool empty() const override;
|
|
||||||
|
|
||||||
void shutdown() override;
|
|
||||||
|
|
||||||
protected:
|
|
||||||
std::vector<DatabasePtr> databases;
|
|
||||||
Poco::Logger * log;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -170,7 +170,7 @@ public:
|
|||||||
/// Get the table for work. Return nullptr if there is no table.
|
/// Get the table for work. Return nullptr if there is no table.
|
||||||
virtual StoragePtr tryGetTable(const String & name, ContextPtr context) const = 0;
|
virtual StoragePtr tryGetTable(const String & name, ContextPtr context) const = 0;
|
||||||
|
|
||||||
virtual StoragePtr getTable(const String & name, ContextPtr context) const;
|
StoragePtr getTable(const String & name, ContextPtr context) const;
|
||||||
|
|
||||||
virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; }
|
virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; }
|
||||||
|
|
||||||
@ -183,8 +183,6 @@ public:
|
|||||||
/// Is the database empty.
|
/// Is the database empty.
|
||||||
virtual bool empty() const = 0;
|
virtual bool empty() const = 0;
|
||||||
|
|
||||||
virtual bool isReadOnly() const { return false; }
|
|
||||||
|
|
||||||
/// Add the table to the database. Record its presence in the metadata.
|
/// Add the table to the database. Record its presence in the metadata.
|
||||||
virtual void createTable(
|
virtual void createTable(
|
||||||
ContextPtr /*context*/,
|
ContextPtr /*context*/,
|
||||||
|
@ -338,17 +338,9 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
|||||||
database = it->second;
|
database = it->second;
|
||||||
}
|
}
|
||||||
|
|
||||||
StoragePtr table = nullptr;
|
auto table = database->tryGetTable(table_id.table_name, context_);
|
||||||
try
|
if (!table && exception)
|
||||||
{
|
exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} doesn't exist", table_id.getNameForLogs()));
|
||||||
table = database->getTable(table_id.table_name, context_);
|
|
||||||
}
|
|
||||||
catch (const Exception & e)
|
|
||||||
{
|
|
||||||
if (exception)
|
|
||||||
exception->emplace(e);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!table)
|
if (!table)
|
||||||
database = nullptr;
|
database = nullptr;
|
||||||
|
|
||||||
|
@ -32,10 +32,5 @@
|
|||||||
<secret_access_key>testtest</secret_access_key>
|
<secret_access_key>testtest</secret_access_key>
|
||||||
<structure>auto</structure>
|
<structure>auto</structure>
|
||||||
</s3_conn>
|
</s3_conn>
|
||||||
<s3_conn_db>
|
|
||||||
<url>http://localhost:11111/test/</url>
|
|
||||||
<access_key_id>test</access_key_id>
|
|
||||||
<secret_access_key>testtest</secret_access_key>
|
|
||||||
</s3_conn_db>
|
|
||||||
</named_collections>
|
</named_collections>
|
||||||
</clickhouse>
|
</clickhouse>
|
||||||
|
@ -1,9 +0,0 @@
|
|||||||
Test 1: check explicit and implicit call of the file table function
|
|
||||||
explicit:
|
|
||||||
4
|
|
||||||
implicit:
|
|
||||||
4
|
|
||||||
Test 2: check Filesystem database
|
|
||||||
4
|
|
||||||
Test 3: check show database with Filesystem
|
|
||||||
test02707
|
|
@ -1,45 +0,0 @@
|
|||||||
#!/usr/bin/env bash
|
|
||||||
|
|
||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|
||||||
# shellcheck source=../shell_config.sh
|
|
||||||
. "$CURDIR"/../shell_config.sh
|
|
||||||
|
|
||||||
dir=${CLICKHOUSE_TEST_UNIQUE_NAME}
|
|
||||||
[[ -d $dir ]] && rm -rd $dir
|
|
||||||
mkdir $dir
|
|
||||||
|
|
||||||
# Create temporary csv file for tests
|
|
||||||
echo '"id","str","int","text"' > $dir/tmp.csv
|
|
||||||
echo '1,"abc",123,"abacaba"' >> $dir/tmp.csv
|
|
||||||
echo '2,"def",456,"bacabaa"' >> $dir/tmp.csv
|
|
||||||
echo '3,"story",78912,"acabaab"' >> $dir/tmp.csv
|
|
||||||
echo '4,"history",21321321,"cabaaba"' >> $dir/tmp.csv
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 1: check explicit and implicit call of the file table function"
|
|
||||||
|
|
||||||
echo "explicit:"
|
|
||||||
$CLICKHOUSE_LOCAL -q "SELECT COUNT(*) FROM file('${dir}/tmp.csv')"
|
|
||||||
echo "implicit:"
|
|
||||||
$CLICKHOUSE_LOCAL -q "SELECT COUNT(*) FROM \"${dir}/tmp.csv\""
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 2: check Filesystem database"
|
|
||||||
$CLICKHOUSE_LOCAL --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test;
|
|
||||||
CREATE DATABASE test ENGINE = Filesystem('${dir}');
|
|
||||||
SELECT COUNT(*) FROM test.\`tmp.csv\`;
|
|
||||||
DROP DATABASE test;
|
|
||||||
"""
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 3: check show database with Filesystem"
|
|
||||||
$CLICKHOUSE_LOCAL --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test02707;
|
|
||||||
CREATE DATABASE test02707 ENGINE = Filesystem('${dir}');
|
|
||||||
SHOW DATABASES;
|
|
||||||
DROP DATABASE test02707;
|
|
||||||
""" | grep "test02707"
|
|
||||||
|
|
||||||
# Remove temporary dir with files
|
|
||||||
rm -rd $dir
|
|
@ -1,15 +0,0 @@
|
|||||||
Test 1: create filesystem database and check implicit calls
|
|
||||||
0
|
|
||||||
test1
|
|
||||||
4
|
|
||||||
4
|
|
||||||
4
|
|
||||||
Test 2: check DatabaseFilesystem access rights and errors handling on server
|
|
||||||
OK
|
|
||||||
OK
|
|
||||||
OK
|
|
||||||
OK
|
|
||||||
OK
|
|
||||||
OK
|
|
||||||
OK
|
|
||||||
OK
|
|
@ -1,72 +0,0 @@
|
|||||||
#!/usr/bin/env bash
|
|
||||||
# Tags: no-parallel
|
|
||||||
|
|
||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|
||||||
# shellcheck source=../shell_config.sh
|
|
||||||
. "$CURDIR"/../shell_config.sh
|
|
||||||
|
|
||||||
# see 01658_read_file_to_stringcolumn.sh
|
|
||||||
CLICKHOUSE_USER_FILES_PATH=$(clickhouse-client --query "select _path, _file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
|
||||||
|
|
||||||
# Prepare data
|
|
||||||
unique_name=${CLICKHOUSE_TEST_UNIQUE_NAME}
|
|
||||||
user_files_tmp_dir=${CLICKHOUSE_USER_FILES_PATH}/${unique_name}
|
|
||||||
mkdir -p ${user_files_tmp_dir}/tmp/
|
|
||||||
echo '"id","str","int","text"' > ${user_files_tmp_dir}/tmp.csv
|
|
||||||
echo '1,"abc",123,"abacaba"' >> ${user_files_tmp_dir}/tmp.csv
|
|
||||||
echo '2,"def",456,"bacabaa"' >> ${user_files_tmp_dir}/tmp.csv
|
|
||||||
echo '3,"story",78912,"acabaab"' >> ${user_files_tmp_dir}/tmp.csv
|
|
||||||
echo '4,"history",21321321,"cabaaba"' >> ${user_files_tmp_dir}/tmp.csv
|
|
||||||
|
|
||||||
tmp_dir=${CLICKHOUSE_TEST_UNIQUE_NAME}
|
|
||||||
[[ -d $tmp_dir ]] && rm -rd $tmp_dir
|
|
||||||
mkdir $tmp_dir
|
|
||||||
cp ${user_files_tmp_dir}/tmp.csv ${tmp_dir}/tmp.csv
|
|
||||||
cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp/tmp.csv
|
|
||||||
cp ${user_files_tmp_dir}/tmp.csv ${user_files_tmp_dir}/tmp.myext
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 1: create filesystem database and check implicit calls"
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test1;
|
|
||||||
CREATE DATABASE test1 ENGINE = Filesystem;
|
|
||||||
"""
|
|
||||||
echo $?
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SHOW DATABASES" | grep "test1"
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.csv\`;"
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp/tmp.csv\`;"
|
|
||||||
${CLICKHOUSE_LOCAL} -q "SELECT COUNT(*) FROM \"${tmp_dir}/tmp.csv\""
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 2: check DatabaseFilesystem access rights and errors handling on server"
|
|
||||||
# DATABASE_ACCESS_DENIED: Allows list files only inside user_files
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK"
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`/tmp/tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK"
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery --query """
|
|
||||||
USE test1;
|
|
||||||
SELECT COUNT(*) FROM \"../${tmp_dir}/tmp.csv\";
|
|
||||||
""" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK"
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`../../../../../../tmp.csv\`;" 2>&1| grep -F "Code: 291" > /dev/null && echo "OK"
|
|
||||||
|
|
||||||
# BAD_ARGUMENTS: path should be inside user_files
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test2;
|
|
||||||
CREATE DATABASE test2 ENGINE = Filesystem('/tmp');
|
|
||||||
""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK"
|
|
||||||
|
|
||||||
# BAD_ARGUMENTS: .../user_files/relative_unknown_dir does not exists
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test2;
|
|
||||||
CREATE DATABASE test2 ENGINE = Filesystem('relative_unknown_dir');
|
|
||||||
""" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK"
|
|
||||||
|
|
||||||
# FILE_DOESNT_EXIST: unknown file
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`tmp2.csv\`;" 2>&1| grep -F "Code: 107" > /dev/null && echo "OK"
|
|
||||||
|
|
||||||
# BAD_ARGUMENTS: Cannot determine the file format by it's extension
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT COUNT(*) FROM test1.\`${unique_name}/tmp.myext\`;" 2>&1| grep -F "Code: 36" > /dev/null && echo "OK"
|
|
||||||
|
|
||||||
# Clean
|
|
||||||
${CLICKHOUSE_CLIENT} --query "DROP DATABASE test1;"
|
|
||||||
rm -rd $tmp_dir
|
|
||||||
rm -rd $user_files_tmp_dir
|
|
@ -1,21 +0,0 @@
|
|||||||
Test 1: select from s3
|
|
||||||
1 2 3
|
|
||||||
4 5 6
|
|
||||||
7 8 9
|
|
||||||
0 0 0
|
|
||||||
test1
|
|
||||||
10 11 12
|
|
||||||
13 14 15
|
|
||||||
16 17 18
|
|
||||||
0 0 0
|
|
||||||
10 11 12
|
|
||||||
13 14 15
|
|
||||||
16 17 18
|
|
||||||
0 0 0
|
|
||||||
10 11 12
|
|
||||||
13 14 15
|
|
||||||
16 17 18
|
|
||||||
0 0 0
|
|
||||||
Test 2: check exceptions
|
|
||||||
OK
|
|
||||||
OK
|
|
@ -1,63 +0,0 @@
|
|||||||
#!/usr/bin/env bash
|
|
||||||
# Tags: no-fasttest, no-parallel
|
|
||||||
# Tag no-fasttest: Depends on AWS
|
|
||||||
|
|
||||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|
||||||
# shellcheck source=../shell_config.sh
|
|
||||||
. "$CUR_DIR"/../shell_config.sh
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 1: select from s3"
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test1;
|
|
||||||
CREATE DATABASE test1 ENGINE = S3;
|
|
||||||
USE test1;
|
|
||||||
SELECT * FROM \"http://localhost:11111/test/a.tsv\"
|
|
||||||
"""
|
|
||||||
${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test1
|
|
||||||
|
|
||||||
# check credentials with absolute path
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test2;
|
|
||||||
CREATE DATABASE test2 ENGINE = S3('', 'test', 'testtest');
|
|
||||||
USE test2;
|
|
||||||
SELECT * FROM \"http://localhost:11111/test/b.tsv\"
|
|
||||||
"""
|
|
||||||
|
|
||||||
# check credentials with relative path
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test4;
|
|
||||||
CREATE DATABASE test4 ENGINE = S3('http://localhost:11111/test', 'test', 'testtest');
|
|
||||||
USE test4;
|
|
||||||
SELECT * FROM \"b.tsv\"
|
|
||||||
"""
|
|
||||||
|
|
||||||
# Check named collection loading
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test5;
|
|
||||||
CREATE DATABASE test5 ENGINE = S3(s3_conn_db);
|
|
||||||
SELECT * FROM test5.\`b.tsv\`
|
|
||||||
"""
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 2: check exceptions"
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test3;
|
|
||||||
CREATE DATABASE test3 ENGINE = S3;
|
|
||||||
USE test3;
|
|
||||||
SELECT * FROM \"http://localhost:11111/test/a.myext\"
|
|
||||||
""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK"
|
|
||||||
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
USE test3;
|
|
||||||
SELECT * FROM \"abacaba\"
|
|
||||||
""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK"
|
|
||||||
|
|
||||||
# Cleanup
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test1;
|
|
||||||
DROP DATABASE IF EXISTS test2;
|
|
||||||
DROP DATABASE IF EXISTS test3;
|
|
||||||
DROP DATABASE IF EXISTS test4;
|
|
||||||
DROP DATABASE IF EXISTS test5;
|
|
||||||
"""
|
|
@ -1,12 +0,0 @@
|
|||||||
Test 1: select from hdfs database
|
|
||||||
1 2 3
|
|
||||||
test1
|
|
||||||
1 2 3
|
|
||||||
test2
|
|
||||||
Test 2: check exceptions
|
|
||||||
OK0
|
|
||||||
OK1
|
|
||||||
OK2
|
|
||||||
OK3
|
|
||||||
OK4
|
|
||||||
OK5
|
|
@ -1,60 +0,0 @@
|
|||||||
#!/usr/bin/env bash
|
|
||||||
# Tags: no-fasttest, use-hdfs, no-parallel
|
|
||||||
|
|
||||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
|
||||||
# shellcheck source=../shell_config.sh
|
|
||||||
. "$CURDIR"/../shell_config.sh
|
|
||||||
|
|
||||||
# Prepare data
|
|
||||||
${CLICKHOUSE_CLIENT} -q "insert into table function hdfs('hdfs://localhost:12222/test_02725_1.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 1, 2, 3 settings hdfs_truncate_on_insert=1;"
|
|
||||||
${CLICKHOUSE_CLIENT} -q "insert into table function hdfs('hdfs://localhost:12222/test_02725_2.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 4, 5, 6 settings hdfs_truncate_on_insert=1;"
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 1: select from hdfs database"
|
|
||||||
|
|
||||||
# Database without specific host
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test1;
|
|
||||||
CREATE DATABASE test1 ENGINE = HDFS;
|
|
||||||
USE test1;
|
|
||||||
SELECT * FROM \"hdfs://localhost:12222/test_02725_1.tsv\"
|
|
||||||
"""
|
|
||||||
${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test1
|
|
||||||
|
|
||||||
# Database with host
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test2;
|
|
||||||
CREATE DATABASE test2 ENGINE = HDFS('hdfs://localhost:12222');
|
|
||||||
USE test2;
|
|
||||||
SELECT * FROM \"test_02725_1.tsv\"
|
|
||||||
"""
|
|
||||||
${CLICKHOUSE_CLIENT} -q "SHOW DATABASES;" | grep test2
|
|
||||||
|
|
||||||
#################
|
|
||||||
echo "Test 2: check exceptions"
|
|
||||||
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test3;
|
|
||||||
CREATE DATABASE test3 ENGINE = HDFS('abacaba');
|
|
||||||
""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK0"
|
|
||||||
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test4;
|
|
||||||
CREATE DATABASE test4 ENGINE = HDFS;
|
|
||||||
USE test4;
|
|
||||||
SELECT * FROM \"abacaba/file.tsv\"
|
|
||||||
""" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK1"
|
|
||||||
|
|
||||||
${CLICKHOUSE_CLIENT} -q "SELECT * FROM test4.\`http://localhost:11111/test/a.tsv\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK2"
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/file.myext\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK3"
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1| grep -F "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK4"
|
|
||||||
${CLICKHOUSE_CLIENT} --query "SELECT * FROM test4.\`hdfs://localhost:12222\`" 2>&1| grep -F "BAD_ARGUMENTS" > /dev/null && echo "OK5"
|
|
||||||
|
|
||||||
|
|
||||||
# Cleanup
|
|
||||||
${CLICKHOUSE_CLIENT} --multiline --multiquery -q """
|
|
||||||
DROP DATABASE IF EXISTS test1;
|
|
||||||
DROP DATABASE IF EXISTS test2;
|
|
||||||
DROP DATABASE IF EXISTS test3;
|
|
||||||
DROP DATABASE IF EXISTS test4;
|
|
||||||
"""
|
|
Loading…
Reference in New Issue
Block a user