mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #5953 from abyss7/fix-filter-table
Implement getDatabaseName() and rename() for all storages
This commit is contained in:
commit
a93c76837e
@ -251,9 +251,10 @@ if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")
|
||||
endif ()
|
||||
|
||||
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
if (NOT SANITIZE)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
endif()
|
||||
|
||||
include (cmake/find_unwind.cmake)
|
||||
|
||||
|
@ -853,7 +853,7 @@ bool TCPHandler::receiveData()
|
||||
if (!(storage = query_context->tryGetExternalTable(external_table_name)))
|
||||
{
|
||||
NamesAndTypesList columns = block.getNamesAndTypesList();
|
||||
storage = StorageMemory::create(external_table_name, ColumnsDescription{columns});
|
||||
storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns});
|
||||
storage->startup();
|
||||
query_context->addExternalTable(external_table_name, storage);
|
||||
}
|
||||
|
@ -160,7 +160,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header,
|
||||
|
||||
/// Create table
|
||||
NamesAndTypesList columns = sample_block.getNamesAndTypesList();
|
||||
StoragePtr storage = StorageMemory::create(data.second, ColumnsDescription{columns});
|
||||
StoragePtr storage = StorageMemory::create("_external", data.second, ColumnsDescription{columns});
|
||||
storage->startup();
|
||||
context.addExternalTable(data.second, storage);
|
||||
BlockOutputStreamPtr output = storage->write(ASTPtr(), context);
|
||||
|
@ -52,7 +52,7 @@ Tables DatabaseDictionary::listTables(const Context & context, const FilterByNam
|
||||
auto dict_name = dict_ptr->getName();
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
tables[dict_name] = StorageDictionary::create(dict_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
}
|
||||
return tables;
|
||||
}
|
||||
@ -73,7 +73,7 @@ StoragePtr DatabaseDictionary::tryGetTable(
|
||||
{
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
return StorageDictionary::create(table_name, ColumnsDescription{columns}, context, true, table_name);
|
||||
return StorageDictionary::create(getDatabaseName(), table_name, ColumnsDescription{columns}, context, true, table_name);
|
||||
}
|
||||
|
||||
return {};
|
||||
|
@ -230,7 +230,7 @@ DatabaseMySQL::MySQLStorageInfo DatabaseMySQL::createStorageInfo(
|
||||
const String & table_name, const NamesAndTypesList & columns_name_and_type, const UInt64 & table_modification_time) const
|
||||
{
|
||||
const auto & mysql_table = StorageMySQL::create(
|
||||
table_name, std::move(mysql_pool), mysql_database_name, table_name,
|
||||
database_name, table_name, std::move(mysql_pool), mysql_database_name, table_name,
|
||||
false, "", ColumnsDescription{columns_name_and_type}, global_context);
|
||||
|
||||
const auto & create_table_query = std::make_shared<ASTCreateQuery>();
|
||||
|
@ -92,7 +92,7 @@ public:
|
||||
Block sample = interpreter->getSampleBlock();
|
||||
NamesAndTypesList columns = sample.getNamesAndTypesList();
|
||||
|
||||
StoragePtr external_storage = StorageMemory::create(external_table_name, ColumnsDescription{columns});
|
||||
StoragePtr external_storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns});
|
||||
external_storage->startup();
|
||||
|
||||
/** We replace the subquery with the name of the temporary table.
|
||||
|
@ -38,6 +38,7 @@ public:
|
||||
std::string getRemoteTableName() const { return remote_table; }
|
||||
|
||||
std::string getTableName() const override { return ""; }
|
||||
std::string getDatabaseName() const override { return ""; }
|
||||
|
||||
protected:
|
||||
StorageDistributedFake(const std::string & remote_database_, const std::string & remote_table_, size_t shard_count_)
|
||||
|
@ -62,7 +62,7 @@ public:
|
||||
|
||||
/// The name of the table.
|
||||
virtual std::string getTableName() const = 0;
|
||||
virtual std::string getDatabaseName() const { return {}; } // FIXME: should be an abstract method!
|
||||
virtual std::string getDatabaseName() const = 0;
|
||||
|
||||
/// Returns true if the storage receives data from a remote server or servers.
|
||||
virtual bool isRemote() const { return false; }
|
||||
|
@ -17,6 +17,7 @@ class StorageFromMergeTreeDataPart : public ext::shared_ptr_helper<StorageFromMe
|
||||
public:
|
||||
String getName() const override { return "FromMergeTreeDataPart"; }
|
||||
String getTableName() const override { return part->storage.getTableName() + " (part " + part->name + ")"; }
|
||||
String getDatabaseName() const override { return part->storage.getDatabaseName(); }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
|
@ -55,17 +55,17 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
StorageBuffer::StorageBuffer(const std::string & name_, const ColumnsDescription & columns_,
|
||||
StorageBuffer::StorageBuffer(const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_,
|
||||
Context & context_,
|
||||
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
|
||||
const String & destination_database_, const String & destination_table_, bool allow_materialized_)
|
||||
: IStorage{columns_},
|
||||
name(name_), global_context(context_),
|
||||
table_name(table_name_), database_name(database_name_), global_context(context_),
|
||||
num_shards(num_shards_), buffers(num_shards_),
|
||||
min_thresholds(min_thresholds_), max_thresholds(max_thresholds_),
|
||||
destination_database(destination_database_), destination_table(destination_table_),
|
||||
no_destination(destination_database.empty() && destination_table.empty()),
|
||||
allow_materialized(allow_materialized_), log(&Logger::get("StorageBuffer (" + name + ")"))
|
||||
allow_materialized(allow_materialized_), log(&Logger::get("StorageBuffer (" + table_name + ")"))
|
||||
{
|
||||
}
|
||||
|
||||
@ -692,7 +692,7 @@ void StorageBuffer::flushThread()
|
||||
}
|
||||
|
||||
|
||||
void StorageBuffer::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context, TableStructureWriteLockHolder & table_lock_holder)
|
||||
void StorageBuffer::alter(const AlterCommands & params, const String & database_name_, const String & table_name_, const Context & context, TableStructureWriteLockHolder & table_lock_holder)
|
||||
{
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
|
||||
@ -702,7 +702,7 @@ void StorageBuffer::alter(const AlterCommands & params, const String & database_
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
@ -741,6 +741,7 @@ void registerStorageBuffer(StorageFactory & factory)
|
||||
UInt64 max_bytes = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), engine_args[8]->as<ASTLiteral &>().value);
|
||||
|
||||
return StorageBuffer::create(
|
||||
args.database_name,
|
||||
args.table_name, args.columns,
|
||||
args.context,
|
||||
num_buckets,
|
||||
|
@ -52,7 +52,8 @@ public:
|
||||
};
|
||||
|
||||
std::string getName() const override { return "Buffer"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
QueryProcessingStage::Enum getQueryProcessingStage(const Context & context) const override;
|
||||
|
||||
@ -71,7 +72,7 @@ public:
|
||||
void shutdown() override;
|
||||
bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override;
|
||||
|
||||
void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override { name = new_table_name; }
|
||||
void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; }
|
||||
|
||||
bool supportsSampling() const override { return true; }
|
||||
bool supportsPrewhere() const override
|
||||
@ -94,7 +95,8 @@ public:
|
||||
const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
|
||||
|
||||
private:
|
||||
String name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
Context global_context;
|
||||
|
||||
@ -138,7 +140,7 @@ protected:
|
||||
/** num_shards - the level of internal parallelism (the number of independent buffers)
|
||||
* The buffer is flushed if all minimum thresholds or at least one of the maximum thresholds are exceeded.
|
||||
*/
|
||||
StorageBuffer(const std::string & name_, const ColumnsDescription & columns_,
|
||||
StorageBuffer(const std::string & database_name_, const std::string & table_name_, const ColumnsDescription & columns_,
|
||||
Context & context_,
|
||||
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
|
||||
const String & destination_database_, const String & destination_table_, bool allow_materialized_);
|
||||
|
@ -90,11 +90,16 @@ static void checkCreationIsAllowed(const String & base_path, const String & path
|
||||
}
|
||||
|
||||
|
||||
StorageCatBoostPool::StorageCatBoostPool(const Context & context,
|
||||
String column_description_file_name_,
|
||||
String data_description_file_name_)
|
||||
: column_description_file_name(std::move(column_description_file_name_)),
|
||||
data_description_file_name(std::move(data_description_file_name_))
|
||||
StorageCatBoostPool::StorageCatBoostPool(
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const Context & context,
|
||||
String column_description_file_name_,
|
||||
String data_description_file_name_)
|
||||
: table_name(table_name_)
|
||||
, database_name(database_name_)
|
||||
, column_description_file_name(std::move(column_description_file_name_))
|
||||
, data_description_file_name(std::move(data_description_file_name_))
|
||||
{
|
||||
auto base_path = canonicalPath(context.getPath());
|
||||
column_description_file_name = resolvePath(base_path, std::move(column_description_file_name));
|
||||
|
@ -11,8 +11,8 @@ class StorageCatBoostPool : public ext::shared_ptr_helper<StorageCatBoostPool>,
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "CatBoostPool"; }
|
||||
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -23,6 +23,7 @@ public:
|
||||
|
||||
private:
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
String column_description_file_name;
|
||||
String data_description_file_name;
|
||||
@ -75,7 +76,7 @@ private:
|
||||
void createSampleBlockAndColumns();
|
||||
|
||||
protected:
|
||||
StorageCatBoostPool(const Context & context, String column_description_file_name, String data_description_file_name);
|
||||
StorageCatBoostPool(const String & database_name_, const String & table_name_, const Context & context, String column_description_file_name, String data_description_file_name);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -24,12 +24,14 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
StorageDictionary::StorageDictionary(
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context,
|
||||
bool attach,
|
||||
const String & dictionary_name_)
|
||||
: IStorage{columns_}, table_name(table_name_),
|
||||
database_name(database_name_),
|
||||
dictionary_name(dictionary_name_),
|
||||
logger(&Poco::Logger::get("StorageDictionary"))
|
||||
{
|
||||
@ -104,7 +106,7 @@ void registerStorageDictionary(StorageFactory & factory)
|
||||
String dictionary_name = args.engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
return StorageDictionary::create(
|
||||
args.table_name, args.columns, args.context, args.attach, dictionary_name);
|
||||
args.database_name, args.table_name, args.columns, args.context, args.attach, dictionary_name);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -24,6 +24,8 @@ class StorageDictionary : public ext::shared_ptr_helper<StorageDictionary>, publ
|
||||
public:
|
||||
std::string getName() const override { return "Dictionary"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
const Context & context,
|
||||
@ -58,13 +60,16 @@ private:
|
||||
using Ptr = MultiVersion<IDictionaryBase>::Version;
|
||||
|
||||
String table_name;
|
||||
String database_name;
|
||||
String dictionary_name;
|
||||
Poco::Logger * logger;
|
||||
|
||||
void checkNamesAndTypesCompatibleWithDictionary(const DictionaryStructure & dictionary_structure) const;
|
||||
|
||||
protected:
|
||||
StorageDictionary(const String & table_name_,
|
||||
StorageDictionary(
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context,
|
||||
bool attach,
|
||||
|
@ -189,7 +189,7 @@ StorageDistributed::StorageDistributed(
|
||||
const ASTPtr & sharding_key_,
|
||||
const String & data_path_,
|
||||
bool attach)
|
||||
: IStorage{columns_}, table_name(table_name),
|
||||
: IStorage{columns_}, table_name(table_name), database_name(database_name),
|
||||
remote_database(remote_database_), remote_table(remote_table_),
|
||||
global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_),
|
||||
sharding_key_expr(sharding_key_ ? buildShardingKeyExpression(sharding_key_, global_context, getColumns().getAllPhysical(), false) : nullptr),
|
||||
@ -343,7 +343,7 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const Context & c
|
||||
|
||||
|
||||
void StorageDistributed::alter(
|
||||
const AlterCommands & params, const String & database_name, const String & current_table_name,
|
||||
const AlterCommands & params, const String & current_database_name, const String & current_table_name,
|
||||
const Context & context, TableStructureWriteLockHolder & table_lock_holder)
|
||||
{
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
@ -351,7 +351,7 @@ void StorageDistributed::alter(
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(current_database_name)->alterTable(context, current_table_name, new_columns, new_indices, {});
|
||||
setColumns(std::move(new_columns));
|
||||
}
|
||||
|
||||
|
@ -52,6 +52,8 @@ public:
|
||||
|
||||
std::string getName() const override { return "Distributed"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
bool supportsSampling() const override { return true; }
|
||||
bool supportsFinal() const override { return true; }
|
||||
bool supportsPrewhere() const override { return true; }
|
||||
@ -79,7 +81,7 @@ public:
|
||||
/// Removes temporary data in local filesystem.
|
||||
void truncate(const ASTPtr &, const Context &) override;
|
||||
|
||||
void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override { table_name = new_table_name; }
|
||||
void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; }
|
||||
/// in the sub-tables, you need to manually add and delete columns
|
||||
/// the structure of the sub-table is not checked
|
||||
void alter(
|
||||
@ -113,6 +115,7 @@ public:
|
||||
ActionLock getActionLock(StorageActionBlockType type) override;
|
||||
|
||||
String table_name;
|
||||
String database_name;
|
||||
String remote_database;
|
||||
String remote_table;
|
||||
ASTPtr remote_table_function_ptr;
|
||||
|
@ -68,12 +68,13 @@ StorageFile::StorageFile(
|
||||
const std::string & table_path_,
|
||||
int table_fd_,
|
||||
const std::string & db_dir_path,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const std::string & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_)
|
||||
: IStorage(columns_),
|
||||
table_name(table_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_)
|
||||
table_name(table_name_), database_name(database_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_)
|
||||
{
|
||||
if (table_fd < 0) /// Will use file
|
||||
{
|
||||
@ -265,7 +266,7 @@ void StorageFile::drop()
|
||||
}
|
||||
|
||||
|
||||
void StorageFile::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name)
|
||||
void StorageFile::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
if (!is_db_table)
|
||||
throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
@ -277,6 +278,8 @@ void StorageFile::rename(const String & new_path_to_db, const String & /*new_dat
|
||||
Poco::File(path).renameTo(path_new);
|
||||
|
||||
path = std::move(path_new);
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
}
|
||||
|
||||
|
||||
@ -327,7 +330,7 @@ void registerStorageFile(StorageFactory & factory)
|
||||
return StorageFile::create(
|
||||
source_path, source_fd,
|
||||
args.data_path,
|
||||
args.table_name, format_name, args.columns,
|
||||
args.database_name, args.table_name, format_name, args.columns,
|
||||
args.context);
|
||||
});
|
||||
}
|
||||
|
@ -21,15 +21,9 @@ class StorageFileBlockOutputStream;
|
||||
class StorageFile : public ext::shared_ptr_helper<StorageFile>, public IStorage
|
||||
{
|
||||
public:
|
||||
std::string getName() const override
|
||||
{
|
||||
return "File";
|
||||
}
|
||||
|
||||
std::string getTableName() const override
|
||||
{
|
||||
return table_name;
|
||||
}
|
||||
std::string getName() const override { return "File"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
@ -62,14 +56,15 @@ protected:
|
||||
const std::string & table_path_,
|
||||
int table_fd_,
|
||||
const std::string & db_dir_path,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const std::string & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_);
|
||||
|
||||
private:
|
||||
|
||||
std::string table_name;
|
||||
std::string database_name;
|
||||
std::string format_name;
|
||||
Context & context_global;
|
||||
|
||||
|
@ -26,6 +26,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
StorageHDFS::StorageHDFS(const String & uri_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
@ -34,6 +35,7 @@ StorageHDFS::StorageHDFS(const String & uri_,
|
||||
, uri(uri_)
|
||||
, format_name(format_name_)
|
||||
, table_name(table_name_)
|
||||
, database_name(database_name_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
@ -144,7 +146,11 @@ BlockInputStreams StorageHDFS::read(
|
||||
max_block_size)};
|
||||
}
|
||||
|
||||
void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {}
|
||||
void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
}
|
||||
|
||||
BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Context & /*context*/)
|
||||
{
|
||||
@ -169,7 +175,7 @@ void registerStorageHDFS(StorageFactory & factory)
|
||||
|
||||
String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
return StorageHDFS::create(url, args.table_name, format_name, args.columns, args.context);
|
||||
return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.context);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -16,15 +16,9 @@ namespace DB
|
||||
class StorageHDFS : public ext::shared_ptr_helper<StorageHDFS>, public IStorage
|
||||
{
|
||||
public:
|
||||
String getName() const override
|
||||
{
|
||||
return "HDFS";
|
||||
}
|
||||
|
||||
String getTableName() const override
|
||||
{
|
||||
return table_name;
|
||||
}
|
||||
String getName() const override { return "HDFS"; }
|
||||
String getTableName() const override { return table_name; }
|
||||
String getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -39,6 +33,7 @@ public:
|
||||
|
||||
protected:
|
||||
StorageHDFS(const String & uri_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
@ -48,6 +43,7 @@ private:
|
||||
String uri;
|
||||
String format_name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
Context & context;
|
||||
|
||||
Logger * log = &Logger::get("StorageHDFS");
|
||||
|
@ -27,7 +27,8 @@ namespace ErrorCodes
|
||||
|
||||
StorageJoin::StorageJoin(
|
||||
const String & path_,
|
||||
const String & name_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const Names & key_names_,
|
||||
bool use_nulls_,
|
||||
SizeLimits limits_,
|
||||
@ -35,7 +36,7 @@ StorageJoin::StorageJoin(
|
||||
ASTTableJoin::Strictness strictness_,
|
||||
const ColumnsDescription & columns_,
|
||||
bool overwrite)
|
||||
: StorageSetOrJoinBase{path_, name_, columns_}
|
||||
: StorageSetOrJoinBase{path_, database_name_, table_name_, columns_}
|
||||
, key_names(key_names_)
|
||||
, use_nulls(use_nulls_)
|
||||
, limits(limits_)
|
||||
@ -160,6 +161,7 @@ void registerStorageJoin(StorageFactory & factory)
|
||||
|
||||
return StorageJoin::create(
|
||||
args.data_path,
|
||||
args.database_name,
|
||||
args.table_name,
|
||||
key_names,
|
||||
join_use_nulls.value,
|
||||
|
@ -57,7 +57,8 @@ private:
|
||||
protected:
|
||||
StorageJoin(
|
||||
const String & path_,
|
||||
const String & name_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const Names & key_names_,
|
||||
bool use_nulls_,
|
||||
SizeLimits limits_,
|
||||
|
@ -211,7 +211,7 @@ Block LogBlockInputStream::readImpl()
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage("while reading column " + name_type.name + " at " + storage.path + escapeForFileName(storage.name));
|
||||
e.addMessage("while reading column " + name_type.name + " at " + storage.path + escapeForFileName(storage.table_name));
|
||||
throw;
|
||||
}
|
||||
|
||||
@ -419,24 +419,25 @@ void LogBlockOutputStream::writeMarks(MarksForColumns && marks)
|
||||
|
||||
StorageLog::StorageLog(
|
||||
const std::string & path_,
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
size_t max_compress_block_size_)
|
||||
: IStorage{columns_},
|
||||
path(path_), name(name_),
|
||||
path(path_), table_name(table_name_), database_name(database_name_),
|
||||
max_compress_block_size(max_compress_block_size_),
|
||||
file_checker(path + escapeForFileName(name) + '/' + "sizes.json")
|
||||
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json")
|
||||
{
|
||||
if (path.empty())
|
||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
/// create files if they do not exist
|
||||
Poco::File(path + escapeForFileName(name) + '/').createDirectories();
|
||||
Poco::File(path + escapeForFileName(table_name) + '/').createDirectories();
|
||||
|
||||
for (const auto & column : getColumns().getAllPhysical())
|
||||
addFiles(column.name, *column.type);
|
||||
|
||||
marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
||||
marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
||||
}
|
||||
|
||||
|
||||
@ -455,7 +456,7 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type)
|
||||
ColumnData & column_data = files[stream_name];
|
||||
column_data.column_index = file_count;
|
||||
column_data.data_file = Poco::File{
|
||||
path + escapeForFileName(name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION};
|
||||
path + escapeForFileName(table_name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION};
|
||||
|
||||
column_names_by_idx.push_back(stream_name);
|
||||
++file_count;
|
||||
@ -508,28 +509,29 @@ void StorageLog::loadMarks()
|
||||
}
|
||||
|
||||
|
||||
void StorageLog::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name)
|
||||
void StorageLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
/// Rename directory with data.
|
||||
Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
||||
Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
||||
|
||||
path = new_path_to_db;
|
||||
name = new_table_name;
|
||||
file_checker.setPath(path + escapeForFileName(name) + '/' + "sizes.json");
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
file_checker.setPath(path + escapeForFileName(table_name) + '/' + "sizes.json");
|
||||
|
||||
for (auto & file : files)
|
||||
file.second.data_file = Poco::File(path + escapeForFileName(name) + '/' + Poco::Path(file.second.data_file.path()).getFileName());
|
||||
file.second.data_file = Poco::File(path + escapeForFileName(table_name) + '/' + Poco::Path(file.second.data_file.path()).getFileName());
|
||||
|
||||
marks_file = Poco::File(path + escapeForFileName(name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
||||
marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
|
||||
}
|
||||
|
||||
void StorageLog::truncate(const ASTPtr &, const Context &)
|
||||
{
|
||||
std::shared_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
String table_dir = path + escapeForFileName(name);
|
||||
String table_dir = path + escapeForFileName(table_name);
|
||||
|
||||
files.clear();
|
||||
file_count = 0;
|
||||
@ -642,7 +644,7 @@ void registerStorageLog(StorageFactory & factory)
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return StorageLog::create(
|
||||
args.data_path, args.table_name, args.columns,
|
||||
args.data_path, args.database_name, args.table_name, args.columns,
|
||||
args.context.getSettings().max_compress_block_size);
|
||||
});
|
||||
}
|
||||
|
@ -24,7 +24,8 @@ friend class LogBlockOutputStream;
|
||||
|
||||
public:
|
||||
std::string getName() const override { return "Log"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
@ -42,7 +43,7 @@ public:
|
||||
|
||||
void truncate(const ASTPtr &, const Context &) override;
|
||||
|
||||
std::string full_path() const { return path + escapeForFileName(name) + '/';}
|
||||
std::string full_path() const { return path + escapeForFileName(table_name) + '/';}
|
||||
|
||||
String getDataPath() const override { return full_path(); }
|
||||
|
||||
@ -53,13 +54,15 @@ protected:
|
||||
*/
|
||||
StorageLog(
|
||||
const std::string & path_,
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
size_t max_compress_block_size_);
|
||||
|
||||
private:
|
||||
String path;
|
||||
String name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
mutable std::shared_mutex rwlock;
|
||||
|
||||
@ -119,7 +122,7 @@ private:
|
||||
*/
|
||||
const Marks & getMarksWithRealRowCount() const;
|
||||
|
||||
std::string getFullPath() const { return path + escapeForFileName(name) + '/'; }
|
||||
std::string getFullPath() const { return path + escapeForFileName(table_name) + '/'; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -297,7 +297,7 @@ static void executeRenameQuery(Context & global_context, const String & database
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedView::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name)
|
||||
void StorageMaterializedView::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
if (has_inner_table && tryGetTargetTable())
|
||||
{
|
||||
@ -313,6 +313,7 @@ void StorageMaterializedView::rename(const String & /*new_path_to_db*/, const St
|
||||
DatabaseAndTableName(database_name, table_name));
|
||||
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
|
||||
global_context.addDependencyUnsafe(
|
||||
DatabaseAndTableName(select_database_name, select_table_name),
|
||||
|
@ -14,6 +14,8 @@ class StorageMaterializedView : public ext::shared_ptr_helper<StorageMaterialize
|
||||
public:
|
||||
std::string getName() const override { return "MaterializedView"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
ASTPtr getInnerQuery() const { return inner_query->clone(); }
|
||||
|
||||
NameAndTypePair getColumn(const String & column_name) const override;
|
||||
|
@ -74,8 +74,8 @@ private:
|
||||
};
|
||||
|
||||
|
||||
StorageMemory::StorageMemory(String table_name_, ColumnsDescription columns_description_)
|
||||
: IStorage{std::move(columns_description_)}, table_name(std::move(table_name_))
|
||||
StorageMemory::StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_)
|
||||
: IStorage{std::move(columns_description_)}, database_name(std::move(database_name_)), table_name(std::move(table_name_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -143,7 +143,7 @@ void registerStorageMemory(StorageFactory & factory)
|
||||
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return StorageMemory::create(args.table_name, args.columns);
|
||||
return StorageMemory::create(args.database_name, args.table_name, args.columns);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -23,8 +23,9 @@ friend class MemoryBlockInputStream;
|
||||
friend class MemoryBlockOutputStream;
|
||||
|
||||
public:
|
||||
std::string getName() const override { return "Memory"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
String getName() const override { return "Memory"; }
|
||||
String getTableName() const override { return table_name; }
|
||||
String getDatabaseName() const override { return database_name; }
|
||||
|
||||
size_t getSize() const { return data.size(); }
|
||||
|
||||
@ -42,9 +43,10 @@ public:
|
||||
|
||||
void truncate(const ASTPtr &, const Context &) override;
|
||||
|
||||
void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override { table_name = new_table_name; }
|
||||
void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; }
|
||||
|
||||
private:
|
||||
String database_name;
|
||||
String table_name;
|
||||
|
||||
/// The data itself. `list` - so that when inserted to the end, the existing iterators are not invalidated.
|
||||
@ -53,7 +55,7 @@ private:
|
||||
std::mutex mutex;
|
||||
|
||||
protected:
|
||||
StorageMemory(String table_name_, ColumnsDescription columns_description_);
|
||||
StorageMemory(String database_name_, String table_name_, ColumnsDescription columns_description_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -46,13 +46,15 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
StorageMerge::StorageMerge(
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & source_database_,
|
||||
const String & table_name_regexp_,
|
||||
const Context & context_)
|
||||
: IStorage(columns_, ColumnsDescription({{"_table", std::make_shared<DataTypeString>()}}, true))
|
||||
, name(name_)
|
||||
, table_name(table_name_)
|
||||
, database_name(database_name_)
|
||||
, source_database(source_database_)
|
||||
, table_name_regexp(table_name_regexp_)
|
||||
, global_context(context_)
|
||||
@ -386,13 +388,13 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr
|
||||
DatabaseIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const
|
||||
{
|
||||
auto database = context.getDatabase(source_database);
|
||||
auto table_name_match = [this](const String & table_name) { return table_name_regexp.match(table_name); };
|
||||
auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); };
|
||||
return database->getIterator(global_context, table_name_match);
|
||||
}
|
||||
|
||||
|
||||
void StorageMerge::alter(
|
||||
const AlterCommands & params, const String & database_name, const String & table_name,
|
||||
const AlterCommands & params, const String & database_name_, const String & table_name_,
|
||||
const Context & context, TableStructureWriteLockHolder & table_lock_holder)
|
||||
{
|
||||
lockStructureExclusively(table_lock_holder, context.getCurrentQueryId());
|
||||
@ -400,7 +402,7 @@ void StorageMerge::alter(
|
||||
auto new_columns = getColumns();
|
||||
auto new_indices = getIndices();
|
||||
params.apply(new_columns);
|
||||
context.getDatabase(database_name)->alterTable(context, table_name, new_columns, new_indices, {});
|
||||
context.getDatabase(database_name_)->alterTable(context, table_name_, new_columns, new_indices, {});
|
||||
setColumns(new_columns);
|
||||
}
|
||||
|
||||
@ -491,7 +493,7 @@ void registerStorageMerge(StorageFactory & factory)
|
||||
String table_name_regexp = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
return StorageMerge::create(
|
||||
args.table_name, args.columns,
|
||||
args.database_name, args.table_name, args.columns,
|
||||
source_database, table_name_regexp, args.context);
|
||||
});
|
||||
}
|
||||
|
@ -16,7 +16,8 @@ class StorageMerge : public ext::shared_ptr_helper<StorageMerge>, public IStorag
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "Merge"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
bool isRemote() const override;
|
||||
|
||||
@ -41,7 +42,7 @@ public:
|
||||
unsigned num_streams) override;
|
||||
|
||||
void drop() override {}
|
||||
void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override { name = new_table_name; }
|
||||
void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override { table_name = new_table_name; database_name = new_database_name; }
|
||||
|
||||
/// you need to add and remove columns in the sub-tables manually
|
||||
/// the structure of sub-tables is not checked
|
||||
@ -52,7 +53,8 @@ public:
|
||||
bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override;
|
||||
|
||||
private:
|
||||
String name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
String source_database;
|
||||
OptimizedRegularExpression table_name_regexp;
|
||||
Context global_context;
|
||||
@ -70,7 +72,8 @@ private:
|
||||
|
||||
protected:
|
||||
StorageMerge(
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & source_database_,
|
||||
const String & table_name_regexp_,
|
||||
|
@ -177,7 +177,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const Context &)
|
||||
clearOldPartsFromFilesystem();
|
||||
}
|
||||
|
||||
void StorageMergeTree::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name)
|
||||
void StorageMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
|
||||
|
||||
@ -185,6 +185,7 @@ void StorageMergeTree::rename(const String & new_path_to_db, const String & /*ne
|
||||
|
||||
path = new_path_to_db;
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
full_path = new_full_path;
|
||||
|
||||
/// NOTE: Logger names are not updated.
|
||||
|
@ -36,7 +36,9 @@ String backQuoteMySQL(const String & x)
|
||||
return res;
|
||||
}
|
||||
|
||||
StorageMySQL::StorageMySQL(const std::string & name,
|
||||
StorageMySQL::StorageMySQL(
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
mysqlxx::Pool && pool,
|
||||
const std::string & remote_database_name,
|
||||
const std::string & remote_table_name,
|
||||
@ -45,7 +47,8 @@ StorageMySQL::StorageMySQL(const std::string & name,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context)
|
||||
: IStorage{columns_}
|
||||
, name(name)
|
||||
, table_name(table_name_)
|
||||
, database_name(database_name_)
|
||||
, remote_database_name(remote_database_name)
|
||||
, remote_table_name(remote_table_name)
|
||||
, replace_query{replace_query}
|
||||
@ -230,6 +233,7 @@ void registerStorageMySQL(StorageFactory & factory)
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
return StorageMySQL::create(
|
||||
args.database_name,
|
||||
args.table_name,
|
||||
std::move(pool),
|
||||
remote_database,
|
||||
|
@ -20,7 +20,8 @@ class StorageMySQL : public ext::shared_ptr_helper<StorageMySQL>, public IStorag
|
||||
{
|
||||
public:
|
||||
StorageMySQL(
|
||||
const std::string & name,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
mysqlxx::Pool && pool,
|
||||
const std::string & remote_database_name,
|
||||
const std::string & remote_table_name,
|
||||
@ -30,7 +31,8 @@ public:
|
||||
const Context & context);
|
||||
|
||||
std::string getName() const override { return "MySQL"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
@ -44,7 +46,8 @@ public:
|
||||
|
||||
private:
|
||||
friend class StorageMySQLBlockOutputStream;
|
||||
std::string name;
|
||||
std::string table_name;
|
||||
std::string database_name;
|
||||
|
||||
std::string remote_database_name;
|
||||
std::string remote_table_name;
|
||||
|
@ -26,7 +26,7 @@ void registerStorageNull(StorageFactory & factory)
|
||||
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return StorageNull::create(args.table_name, args.columns);
|
||||
return StorageNull::create(args.database_name, args.table_name, args.columns);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -19,6 +19,7 @@ class StorageNull : public ext::shared_ptr_helper<StorageNull>, public IStorage
|
||||
public:
|
||||
std::string getName() const override { return "Null"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
@ -36,9 +37,10 @@ public:
|
||||
return std::make_shared<NullBlockOutputStream>(getSampleBlock());
|
||||
}
|
||||
|
||||
void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override
|
||||
void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override
|
||||
{
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
}
|
||||
|
||||
void alter(
|
||||
@ -47,10 +49,11 @@ public:
|
||||
|
||||
private:
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
protected:
|
||||
StorageNull(String table_name_, ColumnsDescription columns_description_)
|
||||
: IStorage{std::move(columns_description_)}, table_name(std::move(table_name_))
|
||||
StorageNull(String database_name_, String table_name_, ColumnsDescription columns_description_)
|
||||
: IStorage{std::move(columns_description_)}, table_name(std::move(table_name_)), database_name(std::move(database_name_))
|
||||
{
|
||||
}
|
||||
};
|
||||
|
@ -88,9 +88,10 @@ BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const
|
||||
|
||||
StorageSetOrJoinBase::StorageSetOrJoinBase(
|
||||
const String & path_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_)
|
||||
: IStorage{columns_}, table_name(table_name_)
|
||||
: IStorage{columns_}, table_name(table_name_), database_name(database_name_)
|
||||
{
|
||||
if (path_.empty())
|
||||
throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
@ -102,9 +103,10 @@ StorageSetOrJoinBase::StorageSetOrJoinBase(
|
||||
|
||||
StorageSet::StorageSet(
|
||||
const String & path_,
|
||||
const String & name_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_)
|
||||
: StorageSetOrJoinBase{path_, name_, columns_},
|
||||
: StorageSetOrJoinBase{path_, database_name_, table_name_, columns_},
|
||||
set(std::make_shared<Set>(SizeLimits(), false))
|
||||
{
|
||||
Block header = getSampleBlock();
|
||||
@ -186,7 +188,7 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path)
|
||||
}
|
||||
|
||||
|
||||
void StorageSetOrJoinBase::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name)
|
||||
void StorageSetOrJoinBase::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
/// Rename directory with data.
|
||||
String new_path = new_path_to_db + escapeForFileName(new_table_name);
|
||||
@ -194,6 +196,7 @@ void StorageSetOrJoinBase::rename(const String & new_path_to_db, const String &
|
||||
|
||||
path = new_path + "/";
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
}
|
||||
|
||||
|
||||
@ -206,7 +209,7 @@ void registerStorageSet(StorageFactory & factory)
|
||||
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return StorageSet::create(args.data_path, args.table_name, args.columns);
|
||||
return StorageSet::create(args.data_path, args.database_name, args.table_name, args.columns);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -20,6 +20,7 @@ class StorageSetOrJoinBase : public IStorage
|
||||
|
||||
public:
|
||||
String getTableName() const override { return table_name; }
|
||||
String getDatabaseName() const override { return database_name; }
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override;
|
||||
|
||||
@ -30,11 +31,13 @@ public:
|
||||
protected:
|
||||
StorageSetOrJoinBase(
|
||||
const String & path_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_);
|
||||
|
||||
String path;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
std::atomic<UInt64> increment = 0; /// For the backup file names.
|
||||
|
||||
@ -76,7 +79,8 @@ private:
|
||||
protected:
|
||||
StorageSet(
|
||||
const String & path_,
|
||||
const String & name_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ColumnsDescription & columns_);
|
||||
};
|
||||
|
||||
|
@ -195,20 +195,21 @@ private:
|
||||
|
||||
StorageStripeLog::StorageStripeLog(
|
||||
const std::string & path_,
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
bool attach,
|
||||
size_t max_compress_block_size_)
|
||||
: IStorage{columns_},
|
||||
path(path_), name(name_),
|
||||
path(path_), table_name(table_name_), database_name(database_name_),
|
||||
max_compress_block_size(max_compress_block_size_),
|
||||
file_checker(path + escapeForFileName(name) + '/' + "sizes.json"),
|
||||
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"),
|
||||
log(&Logger::get("StorageStripeLog"))
|
||||
{
|
||||
if (path.empty())
|
||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
String full_path = path + escapeForFileName(name) + '/';
|
||||
String full_path = path + escapeForFileName(table_name) + '/';
|
||||
if (!attach)
|
||||
{
|
||||
/// create files if they do not exist
|
||||
@ -218,16 +219,17 @@ StorageStripeLog::StorageStripeLog(
|
||||
}
|
||||
|
||||
|
||||
void StorageStripeLog::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name)
|
||||
void StorageStripeLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
/// Rename directory with data.
|
||||
Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
||||
Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
||||
|
||||
path = new_path_to_db;
|
||||
name = new_table_name;
|
||||
file_checker.setPath(path + escapeForFileName(name) + "/" + "sizes.json");
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
file_checker.setPath(path + escapeForFileName(table_name) + "/" + "sizes.json");
|
||||
}
|
||||
|
||||
|
||||
@ -290,16 +292,16 @@ CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Conte
|
||||
|
||||
void StorageStripeLog::truncate(const ASTPtr &, const Context &)
|
||||
{
|
||||
if (name.empty())
|
||||
if (table_name.empty())
|
||||
throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
std::shared_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
auto file = Poco::File(path + escapeForFileName(name));
|
||||
auto file = Poco::File(path + escapeForFileName(table_name));
|
||||
file.remove(true);
|
||||
file.createDirectories();
|
||||
|
||||
file_checker = FileChecker{path + escapeForFileName(name) + '/' + "sizes.json"};
|
||||
file_checker = FileChecker{path + escapeForFileName(table_name) + '/' + "sizes.json"};
|
||||
}
|
||||
|
||||
|
||||
@ -313,7 +315,7 @@ void registerStorageStripeLog(StorageFactory & factory)
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return StorageStripeLog::create(
|
||||
args.data_path, args.table_name, args.columns,
|
||||
args.data_path, args.database_name, args.table_name, args.columns,
|
||||
args.attach, args.context.getSettings().max_compress_block_size);
|
||||
});
|
||||
}
|
||||
|
@ -26,7 +26,8 @@ friend class StripeLogBlockOutputStream;
|
||||
|
||||
public:
|
||||
std::string getName() const override { return "StripeLog"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
@ -49,7 +50,7 @@ public:
|
||||
};
|
||||
using Files_t = std::map<String, ColumnData>;
|
||||
|
||||
std::string full_path() const { return path + escapeForFileName(name) + '/';}
|
||||
std::string full_path() const { return path + escapeForFileName(table_name) + '/';}
|
||||
|
||||
String getDataPath() const override { return full_path(); }
|
||||
|
||||
@ -57,7 +58,8 @@ public:
|
||||
|
||||
private:
|
||||
String path;
|
||||
String name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
size_t max_compress_block_size;
|
||||
|
||||
@ -69,7 +71,8 @@ private:
|
||||
protected:
|
||||
StorageStripeLog(
|
||||
const std::string & path_,
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
bool attach,
|
||||
size_t max_compress_block_size_);
|
||||
|
@ -321,20 +321,21 @@ void TinyLogBlockOutputStream::write(const Block & block)
|
||||
|
||||
StorageTinyLog::StorageTinyLog(
|
||||
const std::string & path_,
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
bool attach,
|
||||
size_t max_compress_block_size_)
|
||||
: IStorage{columns_},
|
||||
path(path_), name(name_),
|
||||
path(path_), table_name(table_name_), database_name(database_name_),
|
||||
max_compress_block_size(max_compress_block_size_),
|
||||
file_checker(path + escapeForFileName(name) + '/' + "sizes.json"),
|
||||
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"),
|
||||
log(&Logger::get("StorageTinyLog"))
|
||||
{
|
||||
if (path.empty())
|
||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
String full_path = path + escapeForFileName(name) + '/';
|
||||
String full_path = path + escapeForFileName(table_name) + '/';
|
||||
if (!attach)
|
||||
{
|
||||
/// create files if they do not exist
|
||||
@ -361,7 +362,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
|
||||
ColumnData column_data;
|
||||
files.insert(std::make_pair(stream_name, column_data));
|
||||
files[stream_name].data_file = Poco::File(
|
||||
path + escapeForFileName(name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
|
||||
path + escapeForFileName(table_name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
|
||||
}
|
||||
};
|
||||
|
||||
@ -370,17 +371,18 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
|
||||
}
|
||||
|
||||
|
||||
void StorageTinyLog::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name)
|
||||
void StorageTinyLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
/// Rename directory with data.
|
||||
Poco::File(path + escapeForFileName(name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
||||
Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
||||
|
||||
path = new_path_to_db;
|
||||
name = new_table_name;
|
||||
file_checker.setPath(path + escapeForFileName(name) + "/" + "sizes.json");
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
file_checker.setPath(path + escapeForFileName(table_name) + "/" + "sizes.json");
|
||||
|
||||
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
|
||||
it->second.data_file = Poco::File(path + escapeForFileName(name) + '/' + Poco::Path(it->second.data_file.path()).getFileName());
|
||||
it->second.data_file = Poco::File(path + escapeForFileName(table_name) + '/' + Poco::Path(it->second.data_file.path()).getFileName());
|
||||
}
|
||||
|
||||
|
||||
@ -412,15 +414,15 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context
|
||||
|
||||
void StorageTinyLog::truncate(const ASTPtr &, const Context &)
|
||||
{
|
||||
if (name.empty())
|
||||
if (table_name.empty())
|
||||
throw Exception("Logical error: table name is empty", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto file = Poco::File(path + escapeForFileName(name));
|
||||
auto file = Poco::File(path + escapeForFileName(table_name));
|
||||
file.remove(true);
|
||||
file.createDirectories();
|
||||
|
||||
files.clear();
|
||||
file_checker = FileChecker{path + escapeForFileName(name) + '/' + "sizes.json"};
|
||||
file_checker = FileChecker{path + escapeForFileName(table_name) + '/' + "sizes.json"};
|
||||
|
||||
for (const auto &column : getColumns().getAllPhysical())
|
||||
addFiles(column.name, *column.type);
|
||||
@ -437,7 +439,7 @@ void registerStorageTinyLog(StorageFactory & factory)
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return StorageTinyLog::create(
|
||||
args.data_path, args.table_name, args.columns,
|
||||
args.data_path, args.database_name, args.table_name, args.columns,
|
||||
args.attach, args.context.getSettings().max_compress_block_size);
|
||||
});
|
||||
}
|
||||
|
@ -25,7 +25,8 @@ friend class TinyLogBlockOutputStream;
|
||||
|
||||
public:
|
||||
std::string getName() const override { return "TinyLog"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
@ -48,7 +49,7 @@ public:
|
||||
};
|
||||
using Files_t = std::map<String, ColumnData>;
|
||||
|
||||
std::string full_path() const { return path + escapeForFileName(name) + '/';}
|
||||
std::string full_path() const { return path + escapeForFileName(table_name) + '/';}
|
||||
|
||||
String getDataPath() const override { return full_path(); }
|
||||
|
||||
@ -56,7 +57,8 @@ public:
|
||||
|
||||
private:
|
||||
String path;
|
||||
String name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
size_t max_compress_block_size;
|
||||
|
||||
@ -72,7 +74,8 @@ private:
|
||||
protected:
|
||||
StorageTinyLog(
|
||||
const std::string & path_,
|
||||
const std::string & name_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
bool attach,
|
||||
size_t max_compress_block_size_);
|
||||
|
@ -26,10 +26,11 @@ namespace ErrorCodes
|
||||
|
||||
IStorageURLBase::IStorageURLBase(const Poco::URI & uri_,
|
||||
const Context & context_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_)
|
||||
: IStorage(columns_), uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_)
|
||||
: IStorage(columns_), uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_), database_name(database_name_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -182,7 +183,11 @@ BlockInputStreams IStorageURLBase::read(const Names & column_names,
|
||||
return {std::make_shared<AddingDefaultsBlockInputStream>(block_input, column_defaults, context)};
|
||||
}
|
||||
|
||||
void IStorageURLBase::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {}
|
||||
void IStorageURLBase::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name)
|
||||
{
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
}
|
||||
|
||||
BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const Context & /*context*/)
|
||||
{
|
||||
@ -209,7 +214,7 @@ void registerStorageURL(StorageFactory & factory)
|
||||
|
||||
String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
return StorageURL::create(uri, args.table_name, format_name, args.columns, args.context);
|
||||
return StorageURL::create(uri, args.database_name, args.table_name, format_name, args.columns, args.context);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -16,10 +16,8 @@ namespace DB
|
||||
class IStorageURLBase : public IStorage
|
||||
{
|
||||
public:
|
||||
String getTableName() const override
|
||||
{
|
||||
return table_name;
|
||||
}
|
||||
String getTableName() const override { return table_name; }
|
||||
String getDatabaseName() const override { return database_name; }
|
||||
|
||||
BlockInputStreams read(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
@ -35,6 +33,7 @@ public:
|
||||
protected:
|
||||
IStorageURLBase(const Poco::URI & uri_,
|
||||
const Context & context_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_);
|
||||
@ -45,6 +44,7 @@ protected:
|
||||
private:
|
||||
String format_name;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
||||
virtual std::string getReadMethod() const;
|
||||
|
||||
@ -67,11 +67,12 @@ class StorageURL : public ext::shared_ptr_helper<StorageURL>, public IStorageURL
|
||||
{
|
||||
public:
|
||||
StorageURL(const Poco::URI & uri_,
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
Context & context_)
|
||||
: IStorageURLBase(uri_, context_, table_name_, format_name_, columns_)
|
||||
: IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -27,10 +27,11 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
StorageView::StorageView(
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns_)
|
||||
: IStorage{columns_}, table_name(table_name_)
|
||||
: IStorage{columns_}, table_name(table_name_), database_name(database_name_)
|
||||
{
|
||||
if (!query.select)
|
||||
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
|
||||
@ -101,7 +102,7 @@ void registerStorageView(StorageFactory & factory)
|
||||
if (args.query.storage)
|
||||
throw Exception("Specifying ENGINE is not allowed for a View", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
return StorageView::create(args.table_name, args.query, args.columns);
|
||||
return StorageView::create(args.database_name, args.table_name, args.query, args.columns);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -15,6 +15,7 @@ class StorageView : public ext::shared_ptr_helper<StorageView>, public IStorage
|
||||
public:
|
||||
std::string getName() const override { return "View"; }
|
||||
std::string getTableName() const override { return table_name; }
|
||||
std::string getDatabaseName() const override { return database_name; }
|
||||
|
||||
/// It is passed inside the query and solved at its level.
|
||||
bool supportsSampling() const override { return true; }
|
||||
@ -28,19 +29,22 @@ public:
|
||||
size_t max_block_size,
|
||||
unsigned num_streams) override;
|
||||
|
||||
void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override
|
||||
void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) override
|
||||
{
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
}
|
||||
|
||||
private:
|
||||
String table_name;
|
||||
String database_name;
|
||||
ASTPtr inner_query;
|
||||
|
||||
void replaceTableNameWithSubquery(ASTSelectQuery * select_query, ASTPtr & subquery);
|
||||
|
||||
protected:
|
||||
StorageView(
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns_);
|
||||
|
@ -22,13 +22,15 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
StorageXDBC::StorageXDBC(const std::string & table_name_,
|
||||
StorageXDBC::StorageXDBC(
|
||||
const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const std::string & remote_database_name_,
|
||||
const std::string & remote_table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const Context & context_,
|
||||
const BridgeHelperPtr bridge_helper_)
|
||||
: IStorageURLBase(Poco::URI(), context_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_)
|
||||
: IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_)
|
||||
, bridge_helper(bridge_helper_)
|
||||
, remote_database_name(remote_database_name_)
|
||||
, remote_table_name(remote_table_name_)
|
||||
@ -116,7 +118,7 @@ namespace
|
||||
BridgeHelperPtr bridge_helper = std::make_shared<XDBCBridgeHelper<BridgeHelperMixin>>(args.context,
|
||||
args.context.getSettingsRef().http_receive_timeout.value,
|
||||
engine_args[0]->as<ASTLiteral &>().value.safeGet<String>());
|
||||
return std::make_shared<StorageXDBC>(args.table_name,
|
||||
return std::make_shared<StorageXDBC>(args.database_name, args.table_name,
|
||||
engine_args[1]->as<ASTLiteral &>().value.safeGet<String>(),
|
||||
engine_args[2]->as<ASTLiteral &>().value.safeGet<String>(),
|
||||
args.columns,
|
||||
|
@ -23,7 +23,8 @@ namespace DB
|
||||
unsigned num_streams) override;
|
||||
|
||||
|
||||
StorageXDBC(const std::string & table_name_,
|
||||
StorageXDBC(const std::string & database_name_,
|
||||
const std::string & table_name_,
|
||||
const std::string & remote_database_name,
|
||||
const std::string & remote_table_name,
|
||||
const ColumnsDescription & columns_,
|
||||
|
@ -25,10 +25,8 @@ public:
|
||||
setColumns(ColumnsDescription(Self::getNamesAndTypes()));
|
||||
}
|
||||
|
||||
std::string getTableName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
BlockInputStreams read(const Names & column_names,
|
||||
const SelectQueryInfo & query_info,
|
||||
|
@ -15,8 +15,8 @@ class StorageSystemColumns : public ext::shared_ptr_helper<StorageSystemColumns>
|
||||
{
|
||||
public:
|
||||
std::string getName() const override { return "SystemColumns"; }
|
||||
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
|
@ -23,6 +23,7 @@ class StorageSystemDetachedParts :
|
||||
public:
|
||||
std::string getName() const override { return "SystemDetachedParts"; }
|
||||
std::string getTableName() const override { return "detached_parts"; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
protected:
|
||||
explicit StorageSystemDetachedParts()
|
||||
|
@ -25,6 +25,7 @@ class StorageSystemNumbers : public ext::shared_ptr_helper<StorageSystemNumbers>
|
||||
public:
|
||||
std::string getName() const override { return "SystemNumbers"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
|
@ -20,6 +20,7 @@ class StorageSystemOne : public ext::shared_ptr_helper<StorageSystemOne>, public
|
||||
public:
|
||||
std::string getName() const override { return "SystemOne"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
|
@ -54,6 +54,7 @@ class StorageSystemPartsBase : public IStorage
|
||||
{
|
||||
public:
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
NameAndTypePair getColumn(const String & column_name) const override;
|
||||
|
||||
|
@ -17,6 +17,7 @@ class StorageSystemReplicas : public ext::shared_ptr_helper<StorageSystemReplica
|
||||
public:
|
||||
std::string getName() const override { return "SystemReplicas"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
|
@ -17,6 +17,7 @@ class StorageSystemTables : public ext::shared_ptr_helper<StorageSystemTables>,
|
||||
public:
|
||||
std::string getName() const override { return "SystemTables"; }
|
||||
std::string getTableName() const override { return name; }
|
||||
std::string getDatabaseName() const override { return "system"; }
|
||||
|
||||
BlockInputStreams read(
|
||||
const Names & column_names,
|
||||
|
@ -23,7 +23,7 @@ struct State
|
||||
{
|
||||
registerFunctions();
|
||||
DatabasePtr database = std::make_shared<DatabaseMemory>("test");
|
||||
database->attachTable("table", StorageMemory::create("table", ColumnsDescription{columns}));
|
||||
database->attachTable("table", StorageMemory::create("test", "table", ColumnsDescription{columns}));
|
||||
context.makeGlobalContext();
|
||||
context.addDatabase("test", database);
|
||||
context.setCurrentDatabase("test");
|
||||
|
@ -25,7 +25,7 @@ try
|
||||
names_and_types.emplace_back("a", std::make_shared<DataTypeUInt64>());
|
||||
names_and_types.emplace_back("b", std::make_shared<DataTypeUInt8>());
|
||||
|
||||
StoragePtr table = StorageLog::create("./", "test", ColumnsDescription{names_and_types}, 1048576);
|
||||
StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, 1048576);
|
||||
table->startup();
|
||||
|
||||
auto context = Context::createGlobal();
|
||||
|
@ -26,6 +26,8 @@ class Context;
|
||||
class ITableFunction
|
||||
{
|
||||
public:
|
||||
static inline std::string getDatabaseName() { return "_table_function"; }
|
||||
|
||||
/// Get the main function name.
|
||||
virtual std::string getName() const = 0;
|
||||
|
||||
|
@ -76,7 +76,7 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co
|
||||
readStringBinary(columns_info, buf);
|
||||
NamesAndTypesList columns = NamesAndTypesList::parse(columns_info);
|
||||
|
||||
auto result = std::make_shared<StorageXDBC>(table_name, schema_name, table_name, ColumnsDescription{columns}, context, helper);
|
||||
auto result = std::make_shared<StorageXDBC>(getDatabaseName(), table_name, schema_name, table_name, ColumnsDescription{columns}, context, helper);
|
||||
|
||||
if (!result)
|
||||
throw Exception("Failed to instantiate storage from table function " + getName(), ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
|
@ -45,7 +45,7 @@ StoragePtr TableFunctionCatBoostPool::executeImpl(const ASTPtr & ast_function, c
|
||||
String column_descriptions_file = getStringLiteral(*args[0], "Column descriptions file");
|
||||
String dataset_description_file = getStringLiteral(*args[1], "Dataset description file");
|
||||
|
||||
return StorageCatBoostPool::create(context, column_descriptions_file, dataset_description_file);
|
||||
return StorageCatBoostPool::create(getDatabaseName(), getName(), context, column_descriptions_file, dataset_description_file);
|
||||
}
|
||||
|
||||
void registerTableFunctionCatBoostPool(TableFunctionFactory & factory)
|
||||
|
@ -10,6 +10,7 @@ StoragePtr TableFunctionFile::getStorage(
|
||||
return StorageFile::create(source,
|
||||
-1,
|
||||
global_context.getUserFilesPath(),
|
||||
getDatabaseName(),
|
||||
getName(),
|
||||
format,
|
||||
ColumnsDescription{sample_block.getNamesAndTypesList()},
|
||||
|
@ -11,6 +11,7 @@ StoragePtr TableFunctionHDFS::getStorage(
|
||||
const String & source, const String & format, const Block & sample_block, Context & global_context) const
|
||||
{
|
||||
return StorageHDFS::create(source,
|
||||
getDatabaseName(),
|
||||
getName(),
|
||||
format,
|
||||
ColumnsDescription{sample_block.getNamesAndTypesList()},
|
||||
|
@ -70,6 +70,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Co
|
||||
String table_name_regexp = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
|
||||
auto res = StorageMerge::create(
|
||||
getDatabaseName(),
|
||||
getName(),
|
||||
ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)},
|
||||
source_database,
|
||||
|
@ -119,6 +119,7 @@ StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & ast_function, const Co
|
||||
throw Exception("MySQL table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
auto res = StorageMySQL::create(
|
||||
getDatabaseName(),
|
||||
table_name,
|
||||
std::move(pool),
|
||||
database_name,
|
||||
|
@ -9,7 +9,7 @@ StoragePtr TableFunctionURL::getStorage(
|
||||
const String & source, const String & format, const Block & sample_block, Context & global_context) const
|
||||
{
|
||||
Poco::URI uri(source);
|
||||
return StorageURL::create(uri, getName(), format, ColumnsDescription{sample_block.getNamesAndTypesList()}, global_context);
|
||||
return StorageURL::create(uri, getDatabaseName(), getName(), format, ColumnsDescription{sample_block.getNamesAndTypesList()}, global_context);
|
||||
}
|
||||
|
||||
void registerTableFunctionURL(TableFunctionFactory & factory)
|
||||
|
Loading…
Reference in New Issue
Block a user