Add commands CREATE/ALTER/DROP NAMED COLLECTION

This commit is contained in:
kssenii 2022-11-11 22:19:58 +01:00
parent 6beab87610
commit 234f6ee54d
27 changed files with 953 additions and 140 deletions

View File

@ -72,6 +72,7 @@ enum class AccessType
\
M(ALTER_TABLE, "", GROUP, ALTER) \
M(ALTER_DATABASE, "", GROUP, ALTER) \
M(ALTER_NAMED_COLLECTION, "", GROUP, ALTER) \
\
M(ALTER_VIEW_REFRESH, "ALTER LIVE VIEW REFRESH, REFRESH VIEW", VIEW, ALTER_VIEW) \
M(ALTER_VIEW_MODIFY_QUERY, "ALTER TABLE MODIFY QUERY", VIEW, ALTER_VIEW) \
@ -88,6 +89,7 @@ enum class AccessType
M(CREATE_TEMPORARY_TABLE, "", GLOBAL, CREATE) /* allows to create and manipulate temporary tables;
implicitly enabled by the grant CREATE_TABLE on any table */ \
M(CREATE_FUNCTION, "", GLOBAL, CREATE) /* allows to execute CREATE FUNCTION */ \
M(CREATE_NAMED_COLLECTION, "", GLOBAL, CREATE) /* allows to execute CREATE NAMED COLLECTION */ \
M(CREATE, "", GROUP, ALL) /* allows to execute {CREATE|ATTACH} */ \
\
M(DROP_DATABASE, "", DATABASE, DROP) /* allows to execute {DROP|DETACH} DATABASE */\
@ -96,6 +98,7 @@ enum class AccessType
implicitly enabled by the grant DROP_TABLE */\
M(DROP_DICTIONARY, "", DICTIONARY, DROP) /* allows to execute {DROP|DETACH} DICTIONARY */\
M(DROP_FUNCTION, "", GLOBAL, DROP) /* allows to execute DROP FUNCTION */\
M(DROP_NAMED_COLLECTION, "", GLOBAL, DROP) /* allows to execute DROP NAMED COLLECTION */\
M(DROP, "", GROUP, ALL) /* allows to execute {DROP|DETACH} */\
\
M(TRUNCATE, "TRUNCATE TABLE", TABLE, ALL) \

View File

@ -639,6 +639,7 @@
M(668, INVALID_STATE) \
M(669, UNKNOWN_NAMED_COLLECTION) \
M(670, NAMED_COLLECTION_ALREADY_EXISTS) \
M(671, NAMED_COLLECTION_IS_IMMUTABLE) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -0,0 +1,79 @@
#include <Interpreters/InterpreterAlterNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Common/FieldVisitorToString.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollections.h>
#include <Common/escapeForFileName.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Parsers/formatAST.h>
#include <Databases/DatabaseOnDisk.h>
namespace DB
{
BlockIO InterpreterAlterNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::ALTER_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTAlterNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
const auto & collection_name = query.collection_name;
const auto & alter_changes = query.changes->as<ASTSetQuery>()->changes;
auto collection = NamedCollectionFactory::instance().getMutable(collection_name);
auto lock = collection->lock();
String collection_name_escaped = escapeForFileName(collection_name);
fs::path metadata_file_path = fs::canonical(getContext()->getPath())
/ "named_collections"
/ (collection_name_escaped + ".sql");
fs::path metadata_file_tmp_path = metadata_file_path.string() + ".tmp";
auto parsed_query = DatabaseOnDisk::parseQueryFromMetadata(
nullptr, getContext(), metadata_file_path, true);
auto & create_named_collection_query = parsed_query->as<ASTCreateNamedCollectionQuery &>();
auto & create_changes = create_named_collection_query.collection_def->as<ASTSetQuery>()->changes;
std::unordered_map<std::string_view, const Field *> new_collection_def;
for (const auto & [name, value] : create_changes)
new_collection_def.emplace(name, &value);
for (const auto & [name, value] : alter_changes)
{
auto it = new_collection_def.find(name);
if (it == new_collection_def.end())
new_collection_def.emplace(name, &value);
else
it->second = &value;
}
WriteBufferFromOwnString statement_buf;
formatAST(query, statement_buf, false);
writeChar('\n', statement_buf);
String statement = statement_buf.str();
WriteBufferFromFile out(metadata_file_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(statement, out);
fs::rename(metadata_file_tmp_path, metadata_file_path);
for (const auto & [name, value] : alter_changes)
collection->setOrUpdate<String>(name, convertFieldToString(value));
return {};
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class Context;
class InterpreterAlterNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterAlterNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_) {}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -0,0 +1,80 @@
#include <Interpreters/InterpreterCreateNamedCollectionQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Common/escapeForFileName.h>
#include <Storages/NamedCollectionConfiguration.h>
#include <Storages/NamedCollections.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Parsers/formatAST.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
BlockIO InterpreterCreateNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::CREATE_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTCreateNamedCollectionQuery &>();
const std::string collection_name = query.collection_name;
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
const ASTSetQuery & collection_def = query.collection_def->as<const ASTSetQuery &>();
const auto config = NamedCollectionConfiguration::createConfiguration(collection_name, collection_def.changes);
NamedCollectionFactory::instance().add(collection_name, NamedCollection::create(*config, collection_name));
String collection_name_escaped = escapeForFileName(collection_name);
fs::path metadata_file_path = fs::canonical(getContext()->getPath())
/ "named_collections"
/ (collection_name_escaped + ".sql");
if (fs::exists(metadata_file_path))
{
NamedCollectionFactory::instance().remove(collection_name);
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Metadata file {} already exists, but according named collection was not loaded");
}
try
{
WriteBufferFromOwnString statement_buf;
formatAST(query, statement_buf, false);
writeChar('\n', statement_buf);
String statement = statement_buf.str();
WriteBufferFromFile out(metadata_file_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(statement, out);
out.next();
if (getContext()->getSettingsRef().fsync_metadata)
out.sync();
out.close();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
NamedCollectionFactory::instance().remove(collection_name);
}
return {};
}
}

View File

@ -0,0 +1,23 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class InterpreterCreateNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterCreateNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_)
{
}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -0,0 +1,32 @@
#include <Interpreters/InterpreterDropNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Access/ContextAccess.h>
#include <Interpreters/Context.h>
#include <Interpreters/executeDDLQueryOnCluster.h>
#include <Storages/NamedCollections.h>
namespace DB
{
BlockIO InterpreterDropNamedCollectionQuery::execute()
{
auto current_context = getContext();
current_context->checkAccess(AccessType::DROP_NAMED_COLLECTION);
const auto & query = query_ptr->as<const ASTDropNamedCollectionQuery &>();
if (!query.cluster.empty())
{
DDLQueryOnClusterParams params;
return executeDDLQueryOnCluster(query_ptr, current_context, params);
}
if (query.if_exists)
NamedCollectionFactory::instance().removeIfExists(query.collection_name);
else
NamedCollectionFactory::instance().remove(query.collection_name);
return {};
}
}

View File

@ -0,0 +1,22 @@
#pragma once
#include <Interpreters/IInterpreter.h>
namespace DB
{
class Context;
class InterpreterDropNamedCollectionQuery : public IInterpreter, WithMutableContext
{
public:
InterpreterDropNamedCollectionQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_)
: WithMutableContext(context_), query_ptr(query_ptr_) {}
BlockIO execute() override;
private:
ASTPtr query_ptr;
};
}

View File

@ -21,6 +21,9 @@
#include <Parsers/ASTShowTablesQuery.h>
#include <Parsers/ASTUseQuery.h>
#include <Parsers/ASTWatchQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Parsers/MySQL/ASTCreateQuery.h>
#include <Parsers/ASTTransactionControl.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
@ -47,6 +50,9 @@
#include <Interpreters/InterpreterCreateFunctionQuery.h>
#include <Interpreters/InterpreterCreateIndexQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/InterpreterCreateNamedCollectionQuery.h>
#include <Interpreters/InterpreterDropNamedCollectionQuery.h>
#include <Interpreters/InterpreterAlterNamedCollectionQuery.h>
#include <Interpreters/InterpreterDeleteQuery.h>
#include <Interpreters/InterpreterDescribeQuery.h>
#include <Interpreters/InterpreterDescribeCacheQuery.h>
@ -230,6 +236,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterAlterQuery>(query, context);
}
else if (query->as<ASTAlterNamedCollectionQuery>())
{
return std::make_unique<InterpreterAlterNamedCollectionQuery>(query, context);
}
else if (query->as<ASTCheckQuery>())
{
return std::make_unique<InterpreterCheckQuery>(query, context);
@ -270,6 +280,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterDropAccessEntityQuery>(query, context);
}
else if (query->as<ASTDropNamedCollectionQuery>())
{
return std::make_unique<InterpreterDropNamedCollectionQuery>(query, context);
}
else if (query->as<ASTGrantQuery>())
{
return std::make_unique<InterpreterGrantQuery>(query, context);
@ -314,6 +328,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
{
return std::make_unique<InterpreterCreateIndexQuery>(query, context);
}
else if (query->as<ASTCreateNamedCollectionQuery>())
{
return std::make_unique<InterpreterCreateNamedCollectionQuery>(query, context);
}
else if (query->as<ASTDropIndexQuery>())
{
return std::make_unique<InterpreterDropIndexQuery>(query, context);

View File

@ -0,0 +1,24 @@
#include <Parsers/ASTAlterNamedCollectionQuery.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
namespace DB
{
ASTPtr ASTAlterNamedCollectionQuery::clone() const
{
auto res = std::make_shared<ASTAlterNamedCollectionQuery>(*this);
res->changes = changes->clone();
return res;
}
void ASTAlterNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "Alter NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " SET " << (settings.hilite ? hilite_none : "");
changes->formatImpl(settings, state, frame);
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
{
class ASTAlterNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
ASTPtr changes;
bool if_exists = false;
String getID(char) const override { return "AlterNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTAlterNamedCollectionQuery>(clone()); }
};
}

View File

@ -0,0 +1,30 @@
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTIdentifier.h>
namespace DB
{
ASTPtr ASTCreateNamedCollectionQuery::clone() const
{
auto res = std::make_shared<ASTCreateNamedCollectionQuery>(*this);
res->collection_def = collection_def->clone();
return res;
}
void ASTCreateNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
collection_def->formatImpl(settings, state, frame);
}
}

View File

@ -0,0 +1,27 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
{
class ASTCreateNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
ASTPtr collection_def;
String getID(char) const override { return "CreateNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTCreateNamedCollectionQuery>(clone()); }
std::string getCollectionName() const;
};
}

View File

@ -0,0 +1,20 @@
#include <Parsers/ASTDropNamedCollectionQuery.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
namespace DB
{
ASTPtr ASTDropNamedCollectionQuery::clone() const
{
return std::make_shared<ASTDropNamedCollectionQuery>(*this);
}
void ASTDropNamedCollectionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP NAMED COLLECTION ";
settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(collection_name) << (settings.hilite ? hilite_none : "");
formatOnCluster(settings);
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOnCluster.h>
namespace DB
{
class ASTDropNamedCollectionQuery : public IAST, public ASTQueryWithOnCluster
{
public:
std::string collection_name;
bool if_exists = false;
String getID(char) const override { return "DropNamedCollectionQuery"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster<ASTDropNamedCollectionQuery>(clone()); }
};
}

View File

@ -0,0 +1,54 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserAlterNamedCollectionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTAlterNamedCollectionQuery.h>
namespace DB
{
bool ParserAlterNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_alter("ALTER");
ParserKeyword s_collection("NAMED COLLECTION");
ParserIdentifier name_p;
ParserSetQuery set_p;
String cluster_str;
bool if_exists = false;
ASTPtr collection_name;
ASTPtr changes;
if (!s_alter.ignore(pos, expected))
return false;
if (!s_collection.ignore(pos, expected))
return false;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
if (!set_p.parse(pos, changes, expected))
return false;
auto query = std::make_shared<ASTAlterNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->if_exists = if_exists;
query->cluster = std::move(cluster_str);
query->changes = changes;
node = query;
return true;
}
}

View File

@ -0,0 +1,14 @@
#pragma once
#include "IParserBase.h"
namespace DB
{
class ParserAlterNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "Alter NAMED COLLECTION query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -9,6 +9,7 @@
#include <Parsers/ASTProjectionDeclaration.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTCreateNamedCollectionQuery.h>
#include <Parsers/ASTTableOverrides.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
@ -1367,6 +1368,50 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
}
bool ParserCreateNamedCollectionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
ParserKeyword s_attach("ATTACH");
ParserKeyword s_named_collection("NAMED COLLECTION");
ParserKeyword s_as("AS");
ParserIdentifier name_p;
ParserSetQuery set_p;
ASTPtr collection_name;
ASTPtr collection_def;
String cluster_str;
if (!s_create.ignore(pos, expected))
return false;
if (!s_named_collection.ignore(pos, expected))
return false;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
if (!s_as.ignore(pos, expected))
return false;
if (!set_p.parse(pos, collection_def, expected))
return false;
auto query = std::make_shared<ASTCreateNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->collection_def = collection_def;
node = query;
return true;
}
bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_create("CREATE");
@ -1478,13 +1523,15 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserCreateDictionaryQuery dictionary_p;
ParserCreateLiveViewQuery live_view_p;
ParserCreateWindowViewQuery window_view_p;
ParserCreateNamedCollectionQuery named_collection_p;
return table_p.parse(pos, node, expected)
|| database_p.parse(pos, node, expected)
|| view_p.parse(pos, node, expected)
|| dictionary_p.parse(pos, node, expected)
|| live_view_p.parse(pos, node, expected)
|| window_view_p.parse(pos, node, expected);
|| window_view_p.parse(pos, node, expected)
|| named_collection_p.parse(pos, node, expected);
}
}

View File

@ -511,6 +511,13 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserCreateNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "CREATE NAMED COLLECTION"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** Query like this:
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name

View File

@ -0,0 +1,50 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ParserDropNamedCollectionQuery.h>
#include <Parsers/ASTDropNamedCollectionQuery.h>
namespace DB
{
bool ParserDropNamedCollectionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
{
ParserKeyword s_drop("DROP");
ParserKeyword s_collection("NAMED COLLECTION");
ParserKeyword s_if_exists("IF EXISTS");
ParserIdentifier name_p;
String cluster_str;
bool if_exists = false;
ASTPtr collection_name;
if (!s_drop.ignore(pos, expected))
return false;
if (!s_collection.ignore(pos, expected))
return false;
if (s_if_exists.ignore(pos, expected))
if_exists = true;
if (!name_p.parse(pos, collection_name, expected))
return false;
if (ParserKeyword{"ON"}.ignore(pos, expected))
{
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
return false;
}
auto query = std::make_shared<ASTDropNamedCollectionQuery>();
tryGetIdentifierNameInto(collection_name, query->collection_name);
query->if_exists = if_exists;
query->cluster = std::move(cluster_str);
node = query;
return true;
}
}

View File

@ -0,0 +1,15 @@
#pragma once
#include "IParserBase.h"
namespace DB
{
class ParserDropNamedCollectionQuery : public IParserBase
{
protected:
const char * getName() const override { return "DROP NAMED COLLECTION query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -5,6 +5,8 @@
#include <Parsers/ParserCreateIndexQuery.h>
#include <Parsers/ParserDropFunctionQuery.h>
#include <Parsers/ParserDropIndexQuery.h>
#include <Parsers/ParserDropNamedCollectionQuery.h>
#include <Parsers/ParserAlterNamedCollectionQuery.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserInsertQuery.h>
#include <Parsers/ParserOptimizeQuery.h>
@ -46,6 +48,8 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserCreateSettingsProfileQuery create_settings_profile_p;
ParserCreateFunctionQuery create_function_p;
ParserDropFunctionQuery drop_function_p;
ParserDropNamedCollectionQuery drop_named_collection_p;
ParserAlterNamedCollectionQuery alter_named_collection_p;
ParserCreateIndexQuery create_index_p;
ParserDropIndexQuery drop_index_p;
ParserDropAccessEntityQuery drop_access_entity_p;
@ -69,6 +73,8 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|| create_settings_profile_p.parse(pos, node, expected)
|| create_function_p.parse(pos, node, expected)
|| drop_function_p.parse(pos, node, expected)
|| drop_named_collection_p.parse(pos, node, expected)
|| alter_named_collection_p.parse(pos, node, expected)
|| create_index_p.parse(pos, node, expected)
|| drop_index_p.parse(pos, node, expected)
|| drop_access_entity_p.parse(pos, node, expected)

View File

@ -0,0 +1,172 @@
#include <Storages/NamedCollectionConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Common/Exception.h>
#include <Common/SettingsChanges.h>
#include <Common/FieldVisitorToString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
}
namespace NamedCollectionConfiguration
{
template <typename T> T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
return getConfigValueOrDefault<T>(config, path);
}
template <typename T> T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value)
{
if (!config.has(path))
{
if (!default_value)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
return *default_value;
}
if constexpr (std::is_same_v<T, String>)
return config.getString(path);
else if constexpr (std::is_same_v<T, UInt64>)
return config.getUInt64(path);
else if constexpr (std::is_same_v<T, Int64>)
return config.getInt64(path);
else if constexpr (std::is_same_v<T, Float64>)
return config.getDouble(path);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in getConfigValueOrDefault(). "
"Supported types are String, UInt64, Int64, Float64");
}
template<typename T> void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update)
{
if (!update && config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path);
if constexpr (std::is_same_v<T, String>)
config.setString(path, value);
else if constexpr (std::is_same_v<T, UInt64>)
config.setUInt64(path, value);
else if constexpr (std::is_same_v<T, Int64>)
config.setInt64(path, value);
else if constexpr (std::is_same_v<T, Float64>)
config.setDouble(path, value);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in setConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
template <typename T> void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path)
{
if (!from_config.has(from_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path);
if (to_config.has(to_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path);
if constexpr (std::is_same_v<T, String>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, std::string>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, UInt64>)
to_config.setUInt64(to_path, from_config.getUInt64(from_path));
else if constexpr (std::is_same_v<T, Int64>)
to_config.setInt64(to_path, from_config.getInt64(from_path));
else if constexpr (std::is_same_v<T, Float64>)
to_config.setDouble(to_path, from_config.getDouble(from_path));
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in copyConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
if (!config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
config.remove(path);
}
ConfigurationPtr createEmptyConfiguration(const std::string & root_name)
{
using DocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
DocumentPtr xml_document(new Poco::XML::Document());
xml_document->appendChild(xml_document->createElement(root_name));
ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document));
return config;
}
ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings)
{
namespace Configuration = NamedCollectionConfiguration;
auto config = Configuration::createEmptyConfiguration(root_name);
for (const auto & [name, value] : settings)
Configuration::setConfigValue<String>(*config, name, convertFieldToString(value));
return config;
}
template String getConfigValue<String>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template UInt64 getConfigValue<UInt64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template Int64 getConfigValue<Int64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template Float64 getConfigValue<Float64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template String getConfigValueOrDefault<String>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const String * default_value);
template UInt64 getConfigValueOrDefault<UInt64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const UInt64 * default_value);
template Int64 getConfigValueOrDefault<Int64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const Int64 * default_value);
template Float64 getConfigValueOrDefault<Float64>(const Poco::Util::AbstractConfiguration & config,
const std::string & path, const Float64 * default_value);
template void setConfigValue<String>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const String & value, bool update);
template void setConfigValue<UInt64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const UInt64 & value, bool update);
template void setConfigValue<Int64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const Int64 & value, bool update);
template void setConfigValue<Float64>(Poco::Util::AbstractConfiguration & config,
const std::string & path, const Float64 & value, bool update);
template void copyConfigValue<String>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<UInt64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<Int64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
template void copyConfigValue<Float64>(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config, const std::string & to_path);
}
}

View File

@ -0,0 +1,44 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
class SettingsChanges;
namespace NamedCollectionConfiguration
{
ConfigurationPtr createEmptyConfiguration(const std::string & root_name);
template <typename T> T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path);
template <typename T> T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value = nullptr);
template<typename T> void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update = false);
template <typename T> void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path);
void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path);
ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings);
}
}

View File

@ -9,9 +9,9 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Storages/NamedCollectionConfiguration.h>
#include <ranges>
@ -22,8 +22,7 @@ namespace ErrorCodes
{
extern const int UNKNOWN_NAMED_COLLECTION;
extern const int NAMED_COLLECTION_ALREADY_EXISTS;
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int NAMED_COLLECTION_IS_IMMUTABLE;
extern const int LOGICAL_ERROR;
}
@ -81,6 +80,9 @@ namespace
}
}
namespace Configuration = NamedCollectionConfiguration;
NamedCollectionFactory & NamedCollectionFactory::instance()
{
static NamedCollectionFactory instance;
@ -164,7 +166,34 @@ NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection
return getImpl(collection_name, lock);
}
NamedCollectionPtr NamedCollectionFactory::getImpl(
MutableNamedCollectionPtr NamedCollectionFactory::getMutable(const std::string & collection_name) const
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
{
throw Exception(
ErrorCodes::UNKNOWN_NAMED_COLLECTION,
"There is no named collection `{}`",
collection_name);
}
auto collection = getImpl(collection_name, lock);
if (!collection->isMutable())
{
/// Collections, which were loaded from a configuration file, cannot be changed.
throw Exception(
ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE,
"Cannot get collection `{}` for modification, "
"because collection was defined as immutable",
collection_name);
}
return collection;
}
MutableNamedCollectionPtr NamedCollectionFactory::getImpl(
const std::string & collection_name,
std::lock_guard<std::mutex> & /* lock */) const
{
@ -180,7 +209,7 @@ NamedCollectionPtr NamedCollectionFactory::getImpl(
void NamedCollectionFactory::add(
const std::string & collection_name,
NamedCollectionPtr collection)
MutableNamedCollectionPtr collection)
{
std::lock_guard lock(mutex);
auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection);
@ -218,6 +247,26 @@ void NamedCollectionFactory::remove(const std::string & collection_name)
assert(removed);
}
void NamedCollectionFactory::removeIfExists(const std::string & collection_name)
{
std::lock_guard lock(mutex);
assertInitialized(lock);
if (!existsUnlocked(collection_name, lock))
return;
if (config->has(collection_name))
{
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Collection {} is defined in config and cannot be removed",
collection_name);
}
[[maybe_unused]] auto removed = loaded_named_collections.erase(collection_name);
assert(removed);
}
NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const
{
std::lock_guard lock(mutex);
@ -242,12 +291,6 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const
class NamedCollection::Impl
{
private:
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/// Named collection configuration
/// <collection1>
/// ...
/// </collection1>
ConfigurationPtr config;
Keys keys;
@ -255,34 +298,34 @@ public:
Impl(const Poco::Util::AbstractConfiguration & config_,
const std::string & collection_name_,
const Keys & keys_)
: config(createEmptyConfiguration(collection_name_))
: config(Configuration::createEmptyConfiguration(collection_name_))
, keys(keys_)
{
auto collection_path = getCollectionPrefix(collection_name_);
for (const auto & key : keys)
copyConfigValue<String>(config_, collection_path + '.' + key, *config, key);
Configuration::copyConfigValue<String>(config_, collection_path + '.' + key, *config, key);
}
template <typename T> T get(const Key & key) const
{
return getConfigValue<T>(*config, key);
return Configuration::getConfigValue<T>(*config, key);
}
template <typename T> T getOrDefault(const Key & key, const T & default_value) const
{
return getConfigValueOrDefault<T>(*config, key, &default_value);
return Configuration::getConfigValueOrDefault<T>(*config, key, &default_value);
}
template <typename T> void set(const Key & key, const T & value, bool update_if_exists)
{
setConfigValue<T>(*config, key, value, update_if_exists);
Configuration::setConfigValue<T>(*config, key, value, update_if_exists);
if (!keys.contains(key))
keys.insert(key);
}
void remove(const Key & key)
{
removeConfigValue(*config, key);
Configuration::removeConfigValue(*config, key);
[[maybe_unused]] auto removed = keys.erase(key);
assert(removed);
}
@ -347,112 +390,6 @@ public:
}
return wb.str();
}
private:
template <typename T> static T getConfigValue(
const Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
return getConfigValueOrDefault<T>(config, path);
}
template <typename T> static T getConfigValueOrDefault(
const Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T * default_value = nullptr)
{
if (!config.has(path))
{
if (!default_value)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
return *default_value;
}
if constexpr (std::is_same_v<T, String>)
return config.getString(path);
else if constexpr (std::is_same_v<T, UInt64>)
return config.getUInt64(path);
else if constexpr (std::is_same_v<T, Int64>)
return config.getInt64(path);
else if constexpr (std::is_same_v<T, Float64>)
return config.getDouble(path);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in getConfigValueOrDefault(). "
"Supported types are String, UInt64, Int64, Float64");
}
template<typename T> static void setConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path,
const T & value,
bool update = false)
{
if (!update && config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path);
if constexpr (std::is_same_v<T, String>)
config.setString(path, value);
else if constexpr (std::is_same_v<T, UInt64>)
config.setUInt64(path, value);
else if constexpr (std::is_same_v<T, Int64>)
config.setInt64(path, value);
else if constexpr (std::is_same_v<T, Float64>)
config.setDouble(path, value);
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in setConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
template <typename T> static void copyConfigValue(
const Poco::Util::AbstractConfiguration & from_config,
const std::string & from_path,
Poco::Util::AbstractConfiguration & to_config,
const std::string & to_path)
{
if (!from_config.has(from_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path);
if (to_config.has(to_path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path);
if constexpr (std::is_same_v<T, String>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, std::string>)
to_config.setString(to_path, from_config.getString(from_path));
else if constexpr (std::is_same_v<T, UInt64>)
to_config.setUInt64(to_path, from_config.getUInt64(from_path));
else if constexpr (std::is_same_v<T, Int64>)
to_config.setInt64(to_path, from_config.getInt64(from_path));
else if constexpr (std::is_same_v<T, Float64>)
to_config.setDouble(to_path, from_config.getDouble(from_path));
else
throw Exception(
ErrorCodes::NOT_IMPLEMENTED,
"Unsupported type in copyConfigValue(). "
"Supported types are String, UInt64, Int64, Float64");
}
static void removeConfigValue(
Poco::Util::AbstractConfiguration & config,
const std::string & path)
{
if (!config.has(path))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path);
config.remove(path);
}
static ConfigurationPtr createEmptyConfiguration(const std::string & root_name)
{
using DocumentPtr = Poco::AutoPtr<Poco::XML::Document>;
DocumentPtr xml_document(new Poco::XML::Document());
xml_document->appendChild(xml_document->createElement(root_name));
ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document));
return config;
}
};
NamedCollection::NamedCollection(
@ -468,7 +405,7 @@ NamedCollection::NamedCollection(ImplPtr pimpl_)
{
}
NamedCollectionPtr NamedCollection::create(
MutableNamedCollectionPtr NamedCollection::create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name)
{
@ -494,39 +431,57 @@ NamedCollectionPtr NamedCollection::create(
template <typename T> T NamedCollection::get(const Key & key) const
{
std::lock_guard lock(mutex);
return pimpl->get<T>(key);
}
template <typename T> T NamedCollection::getOrDefault(const Key & key, const T & default_value) const
{
std::lock_guard lock(mutex);
return pimpl->getOrDefault<T>(key, default_value);
}
template <typename T> void NamedCollection::set(const Key & key, const T & value, bool update_if_exists)
template <typename T> void NamedCollection::set(const Key & key, const T & value)
{
pimpl->set<T>(key, value, update_if_exists);
std::lock_guard lock(mutex);
pimpl->set<T>(key, value, false);
}
template <typename T> void NamedCollection::setOrUpdate(const Key & key, const T & value)
{
std::lock_guard lock(mutex);
pimpl->set<T>(key, value, true);
}
void NamedCollection::remove(const Key & key)
{
std::lock_guard lock(mutex);
pimpl->remove(key);
}
std::shared_ptr<NamedCollection> NamedCollection::duplicate() const
MutableNamedCollectionPtr NamedCollection::duplicate() const
{
std::lock_guard lock(mutex);
return std::make_shared<NamedCollection>(pimpl->copy());
}
NamedCollection::Keys NamedCollection::getKeys() const
{
std::lock_guard lock(mutex);
return pimpl->getKeys();
}
std::string NamedCollection::dumpStructure() const
{
std::lock_guard lock(mutex);
return pimpl->dumpStructure();
}
std::unique_lock<std::mutex> NamedCollection::lock()
{
return std::unique_lock(mutex);
}
template String NamedCollection::get<String>(const NamedCollection::Key & key) const;
template UInt64 NamedCollection::get<UInt64>(const NamedCollection::Key & key) const;
template Int64 NamedCollection::get<Int64>(const NamedCollection::Key & key) const;
@ -537,9 +492,13 @@ template UInt64 NamedCollection::getOrDefault<UInt64>(const NamedCollection::Key
template Int64 NamedCollection::getOrDefault<Int64>(const NamedCollection::Key & key, const Int64 & default_value) const;
template Float64 NamedCollection::getOrDefault<Float64>(const NamedCollection::Key & key, const Float64 & default_value) const;
template void NamedCollection::set<String>(const NamedCollection::Key & key, const String & value, bool update_if_exists);
template void NamedCollection::set<UInt64>(const NamedCollection::Key & key, const UInt64 & value, bool update_if_exists);
template void NamedCollection::set<Int64>(const NamedCollection::Key & key, const Int64 & value, bool update_if_exists);
template void NamedCollection::set<Float64>(const NamedCollection::Key & key, const Float64 & value, bool update_if_exists);
template void NamedCollection::set<String>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::set<UInt64>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::set<Int64>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::set<Float64>(const NamedCollection::Key & key, const Float64 & value);
template void NamedCollection::setOrUpdate<String>(const NamedCollection::Key & key, const String & value);
template void NamedCollection::setOrUpdate<UInt64>(const NamedCollection::Key & key, const UInt64 & value);
template void NamedCollection::setOrUpdate<Int64>(const NamedCollection::Key & key, const Int64 & value);
template void NamedCollection::setOrUpdate<Float64>(const NamedCollection::Key & key, const Float64 & value);
}

View File

@ -9,6 +9,7 @@ namespace DB
class NamedCollection;
using NamedCollectionPtr = std::shared_ptr<const NamedCollection>;
using MutableNamedCollectionPtr = std::shared_ptr<NamedCollection>;
/**
* Class to represent arbitrary-structured named collection object.
@ -27,12 +28,13 @@ private:
using ImplPtr = std::unique_ptr<Impl>;
ImplPtr pimpl;
mutable std::mutex mutex;
public:
using Key = std::string;
using Keys = std::set<Key>;
static NamedCollectionPtr create(
static MutableNamedCollectionPtr create(
const Poco::Util::AbstractConfiguration & config,
const std::string & collection_name);
@ -47,15 +49,21 @@ public:
template <typename T> T getOrDefault(const Key & key, const T & default_value) const;
template <typename T> void set(const Key & key, const T & value, bool update_if_exists = false);
template <typename T> void set(const Key & key, const T & value);
template <typename T> void setOrUpdate(const Key & key, const T & value);
void remove(const Key & key);
std::shared_ptr<NamedCollection> duplicate() const;
bool isMutable() const;
MutableNamedCollectionPtr duplicate() const;
Keys getKeys() const;
std::string dumpStructure() const;
std::unique_lock<std::mutex> lock();
};
/**
@ -76,19 +84,23 @@ public:
NamedCollectionPtr tryGet(const std::string & collection_name) const;
MutableNamedCollectionPtr getMutable(const std::string & collection_name) const;
void add(
const std::string & collection_name,
NamedCollectionPtr collection);
MutableNamedCollectionPtr collection);
void remove(const std::string & collection_name);
using NamedCollections = std::unordered_map<std::string, NamedCollectionPtr>;
void removeIfExists(const std::string & collection_name);
using NamedCollections = std::unordered_map<std::string, MutableNamedCollectionPtr>;
NamedCollections getAll() const;
private:
void assertInitialized(std::lock_guard<std::mutex> & lock) const;
NamedCollectionPtr getImpl(
MutableNamedCollectionPtr getImpl(
const std::string & collection_name,
std::lock_guard<std::mutex> & lock) const;

View File

@ -76,7 +76,7 @@ key5: 5
key6: 6.6
)CONFIG");
collection2_copy->set<String>("key4", "value44", true);
collection2_copy->setOrUpdate<String>("key4", "value44");
ASSERT_TRUE(collection2_copy->get<String>("key4") == "value44");
ASSERT_TRUE(collection2->get<String>("key4") == "value4");
@ -84,7 +84,7 @@ key6: 6.6
ASSERT_TRUE(collection2_copy->getOrDefault<String>("key4", "N") == "N");
ASSERT_TRUE(collection2->getOrDefault<String>("key4", "N") == "value4");
collection2_copy->set<String>("key4", "value45");
collection2_copy->setOrUpdate<String>("key4", "value45");
ASSERT_TRUE(collection2_copy->getOrDefault<String>("key4", "N") == "value45");
NamedCollectionFactory::instance().remove("collection2_copy");