Merge branch 'feature/randomASKII-function' of github.com:BayoNet/ClickHouse into feature/randomASKII-function

This commit is contained in:
BayoNet 2019-11-15 16:30:18 +03:00
commit 2b40692a65
186 changed files with 5081 additions and 1186 deletions

View File

@ -1 +0,0 @@
pr-feature: "New Feature"

19
.github/labeler.yml vendored
View File

@ -1,19 +0,0 @@
# Documentation PRs
documentation:
- "**/*.md"
- "docs/**/*"
pr-documentation:
- "**/*.md"
- "docs/**/*"
# Component labels
comp-mutations:
- "**/*Mutation*"
comp-matview:
- "**/*MaterializedView*"
comp-skipidx:
- "**/*Indices*"
comp-kafka:
- "dbms/src/Storages/Kafka/**/*"
- "dbms/tests/integration/test_storage_kafka/**/*"
- "utils/kafka/**/*"

13
.github/stale.yml vendored
View File

@ -1,7 +1,7 @@
# Configuration for probot-stale - https://github.com/probot/stale
# Number of days of inactivity before an Issue or Pull Request becomes stale
daysUntilStale: 45
daysUntilStale: 60
# Number of days of inactivity before an Issue or Pull Request with the stale label is closed.
# Set to false to disable. If disabled, issues still need to be closed manually, but will remain marked as stale.
@ -21,6 +21,8 @@ exemptLabels:
- st-accepted
- st-in-progress
- st-waiting-for-fix
- segfault
- crash
# Set to true to ignore issues in a project (defaults to false)
exemptProjects: false
@ -37,8 +39,8 @@ staleLabel: stale
# Comment to post when marking as stale. Set to `false` to disable
markComment: >
This issue has been automatically marked as stale because it has not had
recent activity. It will be closed if no further activity occurs. Thank you
for your contributions.
recent activity. Please post a comment if this issue is still relevant to you.
Thank you for your contributions.
# Comment to post when removing the stale label.
# unmarkComment: >
@ -57,10 +59,11 @@ limitPerRun: 30
# Optionally, specify configuration settings that are specific to just 'issues' or 'pulls':
pulls:
daysUntilStale: 365
daysUntilClose: 60
markComment: >
This pull request has been automatically marked as stale because it has not had
any activity for over a year. It will be closed if no further activity occurs. Thank you
for your contributions.
any activity for over a year. Please post a comment about whether you intend to continue working on it.
Thank you for your contributions.
# issues:
# exemptLabels:

View File

@ -1,11 +0,0 @@
name: "Pull Request Labeler"
on:
pull_request
jobs:
by-filename:
runs-on: ubuntu-latest
steps:
- uses: "actions/labeler@v2"
with:
repo-token: "${{ secrets.GITHUB_TOKEN }}"

View File

@ -13,8 +13,9 @@ ClickHouse is an open-source column-oriented database management system that all
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.
## Upcoming Events
* [ClickHouse Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27.
* [ClickHouse Meetup in Tokyo](https://clickhouse.connpass.com/event/147001/) on November 14.
* [ClickHouse Meetup in Istanbul](https://www.eventbrite.com/e/clickhouse-meetup-istanbul-create-blazing-fast-experiences-w-clickhouse-tickets-73101120419) on November 19.
* [ClickHouse Meetup in Ankara](https://www.eventbrite.com/e/clickhouse-meetup-ankara-create-blazing-fast-experiences-w-clickhouse-tickets-73100530655) on November 21.
* [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23.
* [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3.

View File

@ -124,7 +124,7 @@ if (USE_INTERNAL_SSL_LIBRARY)
add_library(OpenSSL::SSL ALIAS ${OPENSSL_SSL_LIBRARY})
endif ()
if (ENABLE_MYSQL AND USE_INTERNAL_MYSQL_LIBRARY)
function(mysql_support)
set(CLIENT_PLUGIN_CACHING_SHA2_PASSWORD STATIC)
set(CLIENT_PLUGIN_SHA256_PASSWORD STATIC)
set(CLIENT_PLUGIN_REMOTE_IO OFF)
@ -136,7 +136,15 @@ if (ENABLE_MYSQL AND USE_INTERNAL_MYSQL_LIBRARY)
if (GLIBC_COMPATIBILITY)
set(LIBM glibc-compatibility)
endif()
if (USE_INTERNAL_ZLIB_LIBRARY)
set(ZLIB_FOUND ON)
set(ZLIB_LIBRARY zlibstatic)
set(WITH_EXTERNAL_ZLIB ON)
endif()
add_subdirectory (mariadb-connector-c)
endfunction()
if (ENABLE_MYSQL AND USE_INTERNAL_MYSQL_LIBRARY)
mysql_support()
endif ()
if (USE_INTERNAL_RDKAFKA_LIBRARY)

View File

@ -130,8 +130,8 @@ list (APPEND dbms_headers
list (APPEND dbms_sources src/TableFunctions/ITableFunction.cpp src/TableFunctions/TableFunctionFactory.cpp)
list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctions/TableFunctionFactory.h)
list (APPEND dbms_sources src/Dictionaries/DictionaryFactory.cpp src/Dictionaries/DictionarySourceFactory.cpp src/Dictionaries/DictionaryStructure.cpp)
list (APPEND dbms_headers src/Dictionaries/DictionaryFactory.h src/Dictionaries/DictionarySourceFactory.h src/Dictionaries/DictionaryStructure.h)
list (APPEND dbms_sources src/Dictionaries/DictionaryFactory.cpp src/Dictionaries/DictionarySourceFactory.cpp src/Dictionaries/DictionaryStructure.cpp src/Dictionaries/getDictionaryConfigurationFromAST.cpp)
list (APPEND dbms_headers src/Dictionaries/DictionaryFactory.h src/Dictionaries/DictionarySourceFactory.h src/Dictionaries/DictionaryStructure.h src/Dictionaries/getDictionaryConfigurationFromAST.h)
if (NOT ENABLE_SSL)
list (REMOVE_ITEM clickhouse_common_io_sources src/Common/OpenSSLHelpers.cpp)

View File

@ -365,7 +365,7 @@ private:
Stopwatch watch;
RemoteBlockInputStream stream(
*(*connection_entries[connection_index]),
query, {}, global_context, &settings, nullptr, Tables(), query_processing_stage);
query, {}, global_context, &settings, nullptr, Scalars(), Tables(), query_processing_stage);
Progress progress;
stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });

View File

@ -38,12 +38,16 @@ namespace
switch (type)
{
case SQL_TINYINT:
return factory.get("Int8");
case SQL_INTEGER:
return factory.get("Int32");
case SQL_SMALLINT:
return factory.get("Int16");
case SQL_BIGINT:
return factory.get("Int64");
case SQL_FLOAT:
return factory.get("Float32");
return factory.get("Float64");
case SQL_REAL:
return factory.get("Float32");
case SQL_DOUBLE:

View File

@ -44,7 +44,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
if (db.second->getEngineName() == "Lazy")
continue;
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
{
auto & table = iterator->table();
StorageReplicatedMergeTree * table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());

View File

@ -37,10 +37,12 @@
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Interpreters/SystemLog.cpp>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
@ -920,6 +922,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->tryCreateEmbeddedDictionaries();
global_context->getExternalDictionariesLoader().enableAlwaysLoadEverything(true);
}
auto dictionaries_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "dictionaries_config");
global_context->getExternalDictionariesLoader().addConfigRepository("", std::move(dictionaries_repository));
auto models_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "models_config");
global_context->getExternalModelsLoader().addConfigRepository("", std::move(models_repository));
}
catch (...)
{

View File

@ -850,9 +850,10 @@ bool TCPHandler::receivePacket()
return true;
case Protocol::Client::Data:
case Protocol::Client::Scalar:
if (state.empty())
receiveUnexpectedData();
return receiveData();
return receiveData(packet_type == Protocol::Client::Scalar);
case Protocol::Client::Ping:
writeVarUInt(Protocol::Server::Pong, *out);
@ -957,39 +958,44 @@ void TCPHandler::receiveUnexpectedQuery()
throw NetException("Unexpected packet Query received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}
bool TCPHandler::receiveData()
bool TCPHandler::receiveData(bool scalar)
{
initBlockInput();
/// The name of the temporary table for writing data, default to empty string
String external_table_name;
readStringBinary(external_table_name, *in);
String name;
readStringBinary(name, *in);
/// Read one block from the network and write it down
Block block = state.block_in->read();
if (block)
{
/// If there is an insert request, then the data should be written directly to `state.io.out`.
/// Otherwise, we write the blocks in the temporary `external_table_name` table.
if (!state.need_receive_data_for_insert && !state.need_receive_data_for_input)
{
StoragePtr storage;
/// If such a table does not exist, create it.
if (!(storage = query_context->tryGetExternalTable(external_table_name)))
{
NamesAndTypesList columns = block.getNamesAndTypesList();
storage = StorageMemory::create("_external", external_table_name, ColumnsDescription{columns}, ConstraintsDescription{});
storage->startup();
query_context->addExternalTable(external_table_name, storage);
}
/// The data will be written directly to the table.
state.io.out = storage->write(ASTPtr(), *query_context);
}
if (state.need_receive_data_for_input)
state.block_for_input = block;
if (scalar)
query_context->addScalar(name, block);
else
state.io.out->write(block);
{
/// If there is an insert request, then the data should be written directly to `state.io.out`.
/// Otherwise, we write the blocks in the temporary `external_table_name` table.
if (!state.need_receive_data_for_insert && !state.need_receive_data_for_input)
{
StoragePtr storage;
/// If such a table does not exist, create it.
if (!(storage = query_context->tryGetExternalTable(name)))
{
NamesAndTypesList columns = block.getNamesAndTypesList();
storage = StorageMemory::create("_external", name, ColumnsDescription{columns}, ConstraintsDescription{});
storage->startup();
query_context->addExternalTable(name, storage);
}
/// The data will be written directly to the table.
state.io.out = storage->write(ASTPtr(), *query_context);
}
if (state.need_receive_data_for_input)
state.block_for_input = block;
else
state.io.out->write(block);
}
return true;
}
else

View File

@ -153,7 +153,7 @@ private:
void receiveHello();
bool receivePacket();
void receiveQuery();
bool receiveData();
bool receiveData(bool scalar);
bool readDataNext(const size_t & poll_interval, const int & receive_timeout);
void readData(const Settings & global_settings);
std::tuple<size_t, int> getReadTimeouts(const Settings & global_settings);

View File

@ -180,7 +180,21 @@
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_localhost>
</test_cluster_two_shards_localhost>
<test_cluster_two_shards>
<shard>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards>
<test_shard_localhost_secure>
<shard>
<replica>

View File

@ -30,6 +30,7 @@
namespace CurrentMetrics
{
extern const Metric SendScalars;
extern const Metric SendExternalTables;
}
@ -441,7 +442,7 @@ void Connection::sendCancel()
}
void Connection::sendData(const Block & block, const String & name)
void Connection::sendData(const Block & block, const String & name, bool scalar)
{
//LOG_TRACE(log_wrapper.get(), "Sending data");
@ -455,7 +456,10 @@ void Connection::sendData(const Block & block, const String & name)
block_out = std::make_shared<NativeBlockOutputStream>(*maybe_compressed_out, server_revision, block.cloneEmpty());
}
writeVarUInt(Protocol::Client::Data, *out);
if (scalar)
writeVarUInt(Protocol::Client::Scalar, *out);
else
writeVarUInt(Protocol::Client::Data, *out);
writeStringBinary(name, *out);
size_t prev_bytes = out->count();
@ -484,6 +488,44 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String
}
void Connection::sendScalarsData(Scalars & data)
{
if (data.empty())
return;
Stopwatch watch;
size_t out_bytes = out ? out->count() : 0;
size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0;
size_t rows = 0;
CurrentMetrics::Increment metric_increment{CurrentMetrics::SendScalars};
for (auto & elem : data)
{
rows += elem.second.rows();
sendData(elem.second, elem.first, true /* scalar */);
}
out_bytes = out->count() - out_bytes;
maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes;
double elapsed = watch.elapsedSeconds();
std::stringstream msg;
msg << std::fixed << std::setprecision(3);
msg << "Sent data for " << data.size() << " scalars, total " << rows << " rows in " << elapsed << " sec., "
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., "
<< maybe_compressed_out_bytes / 1048576.0 << " MiB (" << maybe_compressed_out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
if (compression == Protocol::Compression::Enable)
msg << ", compressed " << static_cast<double>(maybe_compressed_out_bytes) / out_bytes << " times to "
<< out_bytes / 1048576.0 << " MiB (" << out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
else
msg << ", no compression.";
LOG_DEBUG(log_wrapper.get(), msg.rdbuf());
}
void Connection::sendExternalTablesData(ExternalTablesData & data)
{
if (data.empty())

View File

@ -133,7 +133,9 @@ public:
void sendCancel();
/// Send block of data; if name is specified, server will write it to external (temporary) table of that name.
void sendData(const Block & block, const String & name = "");
void sendData(const Block & block, const String & name = "", bool scalar = false);
/// Send all scalars.
void sendScalarsData(Scalars & data);
/// Send all contents of external (temporary) tables.
void sendExternalTablesData(ExternalTablesData & data);

View File

@ -51,6 +51,21 @@ MultiplexedConnections::MultiplexedConnections(
active_connection_count = connections.size();
}
void MultiplexedConnections::sendScalarsData(Scalars & data)
{
std::lock_guard lock(cancel_mutex);
if (!sent_query)
throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
for (ReplicaState & state : replica_states)
{
Connection * connection = state.connection;
if (connection != nullptr)
connection->sendScalarsData(data);
}
}
void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
{
std::lock_guard lock(cancel_mutex);

View File

@ -27,6 +27,8 @@ public:
std::vector<IConnectionPool::Entry> && connections,
const Settings & settings_, const ThrottlerPtr & throttler_);
/// Send all scalars to replicas.
void sendScalarsData(Scalars & data);
/// Send all content of external tables to replicas.
void sendExternalTablesData(std::vector<ExternalTablesData> & data);

View File

@ -21,6 +21,7 @@
M(OpenFileForWrite, "Number of files open for writing") \
M(Read, "Number of read (read, pread, io_getevents, etc.) syscalls in fly") \
M(Write, "Number of write (write, pwrite, io_getevents, etc.) syscalls in fly") \
M(SendScalars, "Number of connections that are sending data for scalars to remote servers.") \
M(SendExternalTables, "Number of connections that are sending data for external tables to remote servers. External tables are used to implement GLOBAL IN and GLOBAL JOIN operators with distributed subqueries.") \
M(QueryThread, "Number of query processing threads") \
M(ReadonlyReplica, "Number of Replicated tables that are currently in readonly state due to re-initialization after ZooKeeper session loss or due to startup without ZooKeeper configured.") \

View File

@ -155,7 +155,7 @@ namespace ErrorCodes
extern const int NOT_FOUND_FUNCTION_ELEMENT_FOR_AGGREGATE = 147;
extern const int NOT_FOUND_RELATION_ELEMENT_FOR_CONDITION = 148;
extern const int NOT_FOUND_RHS_ELEMENT_FOR_CONDITION = 149;
extern const int NO_ATTRIBUTES_LISTED = 150;
extern const int EMPTY_LIST_OF_ATTRIBUTES_PASSED = 150;
extern const int INDEX_OF_COLUMN_IN_SORT_CLAUSE_IS_OUT_OF_RANGE = 151;
extern const int UNKNOWN_DIRECTION_OF_SORTING = 152;
extern const int ILLEGAL_DIVISION = 153;
@ -361,7 +361,7 @@ namespace ErrorCodes
extern const int PART_IS_TEMPORARILY_LOCKED = 384;
extern const int MULTIPLE_STREAMS_REQUIRED = 385;
extern const int NO_COMMON_TYPE = 386;
extern const int EXTERNAL_LOADABLE_ALREADY_EXISTS = 387;
extern const int DICTIONARY_ALREADY_EXISTS = 387;
extern const int CANNOT_ASSIGN_OPTIMIZE = 388;
extern const int INSERT_WAS_DEDUPLICATED = 389;
extern const int CANNOT_GET_CREATE_TABLE_QUERY = 390;
@ -459,6 +459,11 @@ namespace ErrorCodes
extern const int DICTIONARY_ACCESS_DENIED = 482;
extern const int TOO_MANY_REDIRECTS = 483;
extern const int INTERNAL_REDIS_ERROR = 484;
extern const int SCALAR_ALREADY_EXISTS = 485;
extern const int UNKNOWN_SCALAR = 486;
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY = 487;
extern const int UNKNOWN_DICTIONARY = 488;
extern const int INCORRECT_DICTIONARY_DEFINITION = 489;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -68,7 +68,7 @@ protected:
public:
using Configuration = Poco::Util::AbstractConfiguration;
Context & context;
const Context & context;
const Configuration & config;
static constexpr inline auto DEFAULT_HOST = "localhost";
@ -79,7 +79,7 @@ public:
static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote";
static constexpr inline auto PING_OK_ANSWER = "Ok.";
XDBCBridgeHelper(Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_)
XDBCBridgeHelper(const Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_)
: http_timeout(http_timeout_), connection_string(connection_string_), context(global_context_), config(context.getConfigRef())
{
size_t bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT);

View File

@ -112,7 +112,8 @@ namespace Protocol
Cancel = 3, /// Cancel the query execution.
Ping = 4, /// Check that connection to the server is alive.
TablesStatusRequest = 5, /// Check status of tables on the server.
KeepAlive = 6 /// Keep the connection alive
KeepAlive = 6, /// Keep the connection alive
Scalar = 7 /// A block of data (compressed or not).
};
inline const char * toString(UInt64 packet)

View File

@ -379,6 +379,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \
M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.") \
\
M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.") \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\
M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \

View File

@ -23,8 +23,8 @@ namespace ErrorCodes
RemoteBlockInputStream::RemoteBlockInputStream(
Connection & connection,
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{
if (settings)
context.setSettings(*settings);
@ -38,8 +38,8 @@ RemoteBlockInputStream::RemoteBlockInputStream(
RemoteBlockInputStream::RemoteBlockInputStream(
std::vector<IConnectionPool::Entry> && connections,
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{
if (settings)
context.setSettings(*settings);
@ -54,8 +54,8 @@ RemoteBlockInputStream::RemoteBlockInputStream(
RemoteBlockInputStream::RemoteBlockInputStream(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{
if (settings)
context.setSettings(*settings);
@ -120,6 +120,11 @@ void RemoteBlockInputStream::cancel(bool kill)
tryCancel("Cancelling query");
}
void RemoteBlockInputStream::sendScalars()
{
multiplexed_connections->sendScalarsData(scalars);
}
void RemoteBlockInputStream::sendExternalTables()
{
size_t count = multiplexed_connections->size();
@ -308,6 +313,8 @@ void RemoteBlockInputStream::sendQuery()
established = false;
sent_query = true;
if (settings.enable_scalar_subquery_optimization)
sendScalars();
sendExternalTables();
}

View File

@ -25,7 +25,7 @@ public:
RemoteBlockInputStream(
Connection & connection,
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
/// Accepts several connections already taken from pool.
@ -33,7 +33,7 @@ public:
RemoteBlockInputStream(
std::vector<IConnectionPool::Entry> && connections,
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
/// Takes a pool and gets one or several connections from it.
@ -41,7 +41,7 @@ public:
RemoteBlockInputStream(
const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
~RemoteBlockInputStream() override;
@ -71,6 +71,9 @@ public:
Block getHeader() const override { return header; }
protected:
/// Send all scalars to remote servers
void sendScalars();
/// Send all temporary tables to remote servers
void sendExternalTables();
@ -103,6 +106,8 @@ private:
String query_id = "";
Context context;
/// Scalars needed to be sent to remote servers
Scalars scalars;
/// Temporary tables needed to be sent to remote servers
Tables external_tables;
QueryProcessingStage::Enum stage;

View File

@ -19,6 +19,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int SYNTAX_ERROR;
extern const int UNSUPPORTED_METHOD;
}
DatabaseDictionary::DatabaseDictionary(const String & name_)
@ -27,32 +28,36 @@ DatabaseDictionary::DatabaseDictionary(const String & name_)
{
}
void DatabaseDictionary::loadTables(Context &, bool)
void DatabaseDictionary::loadStoredObjects(Context &, bool)
{
}
Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name)
{
Tables tables;
ExternalLoader::Loadables loadables;
ExternalLoader::LoadResults load_results;
if (filter_by_name)
{
/// If `filter_by_name` is set, we iterate through all dictionaries with such names. That's why we need to load all of them.
loadables = context.getExternalDictionariesLoader().loadAndGet(filter_by_name);
context.getExternalDictionariesLoader().load(filter_by_name, load_results);
}
else
{
/// If `filter_by_name` isn't set, we iterate through only already loaded dictionaries. We don't try to load all dictionaries in this case.
loadables = context.getExternalDictionariesLoader().getCurrentlyLoadedObjects();
load_results = context.getExternalDictionariesLoader().getCurrentLoadResults();
}
for (const auto & loadable : loadables)
for (const auto & [object_name, info]: load_results)
{
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(loadable);
auto dict_name = dict_ptr->getName();
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name);
/// Load tables only from XML dictionaries, don't touch other
if (info.object != nullptr && info.repository_name.empty())
{
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(info.object);
auto dict_name = dict_ptr->getName();
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name);
}
}
return tables;
}
@ -64,6 +69,66 @@ bool DatabaseDictionary::isTableExist(
return context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
}
bool DatabaseDictionary::isDictionaryExist(
const Context & /*context*/,
const String & /*table_name*/) const
{
return false;
}
DatabaseDictionariesIteratorPtr DatabaseDictionary::getDictionariesIterator(
const Context & /*context*/,
const FilterByNameFunction & /*filter_by_dictionary_name*/)
{
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
}
void DatabaseDictionary::createDictionary(
const Context & /*context*/,
const String & /*dictionary_name*/,
const ASTPtr & /*query*/)
{
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseDictionary::removeDictionary(
const Context & /*context*/,
const String & /*table_name*/)
{
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseDictionary::attachDictionary(
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
{
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseDictionary::detachDictionary(
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
{
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
}
ASTPtr DatabaseDictionary::tryGetCreateDictionaryQuery(
const Context & /*context*/,
const String & /*table_name*/) const
{
return nullptr;
}
ASTPtr DatabaseDictionary::getCreateDictionaryQuery(
const Context & /*context*/,
const String & /*table_name*/) const
{
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
}
StoragePtr DatabaseDictionary::tryGetTable(
const Context & context,
const String & table_name) const
@ -79,9 +144,9 @@ StoragePtr DatabaseDictionary::tryGetTable(
return {};
}
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context, const FilterByNameFunction & filter_by_name)
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name)
{
return std::make_unique<DatabaseSnapshotIterator>(listTables(context, filter_by_name));
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(context, filter_by_name));
}
bool DatabaseDictionary::empty(const Context & context) const
@ -115,7 +180,7 @@ void DatabaseDictionary::removeTable(
throw Exception("DatabaseDictionary: removeTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
}
time_t DatabaseDictionary::getTableMetadataModificationTime(
time_t DatabaseDictionary::getObjectMetadataModificationTime(
const Context &,
const String &)
{

View File

@ -31,7 +31,7 @@ public:
return "Dictionary";
}
void loadTables(
void loadStoredObjects(
Context & context,
bool has_force_restore_data_flag) override;
@ -39,11 +39,15 @@ public:
const Context & context,
const String & table_name) const override;
bool isDictionaryExist(const Context & context, const String & table_name) const override;
StoragePtr tryGetTable(
const Context & context,
const String & table_name) const override;
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
bool empty(const Context & context) const override;
@ -53,14 +57,20 @@ public:
const StoragePtr & table,
const ASTPtr & query) override;
void createDictionary(
const Context & context, const String & dictionary_name, const ASTPtr & query) override;
void removeTable(
const Context & context,
const String & table_name) override;
void removeDictionary(const Context & context, const String & table_name) override;
void attachTable(const String & table_name, const StoragePtr & table) override;
StoragePtr detachTable(const String & table_name) override;
time_t getTableMetadataModificationTime(
time_t getObjectMetadataModificationTime(
const Context & context,
const String & table_name) override;
@ -74,6 +84,15 @@ public:
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
ASTPtr getCreateDictionaryQuery(const Context & context, const String & table_name) const override;
ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & table_name) const override;
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void shutdown() override;
private:

View File

@ -53,11 +53,13 @@ DatabasePtr DatabaseFactory::get(
else if (engine_name == "MySQL")
{
const ASTFunction * engine = engine_define->engine;
const auto & arguments = engine->arguments->children;
if (arguments.size() != 4)
throw Exception("MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.",
ErrorCodes::BAD_ARGUMENTS);
if (!engine->arguments || engine->arguments->children.size() != 4)
throw Exception(
"MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.",
ErrorCodes::BAD_ARGUMENTS);
const auto & arguments = engine->arguments->children;
const auto & mysql_host_name = arguments[0]->as<ASTLiteral>()->value.safeGet<String>();
const auto & mysql_database_name = arguments[1]->as<ASTLiteral>()->value.safeGet<String>();
@ -74,11 +76,11 @@ DatabasePtr DatabaseFactory::get(
else if (engine_name == "Lazy")
{
const ASTFunction * engine = engine_define->engine;
const auto & arguments = engine->arguments->children;
if (arguments.size() != 1)
throw Exception("Lazy database require cache_expiration_time_seconds argument.",
ErrorCodes::BAD_ARGUMENTS);
if (!engine->arguments || engine->arguments->children.size() != 1)
throw Exception("Lazy database require cache_expiration_time_seconds argument", ErrorCodes::BAD_ARGUMENTS);
const auto & arguments = engine->arguments->children;
const auto cache_expiration_time_seconds = arguments[0]->as<ASTLiteral>()->value.safeGet<UInt64>();
return std::make_shared<DatabaseLazy>(database_name, metadata_path, cache_expiration_time_seconds, context);

View File

@ -29,8 +29,6 @@ namespace ErrorCodes
}
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context)
: name(name_)
@ -43,11 +41,11 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
}
void DatabaseLazy::loadTables(
void DatabaseLazy::loadStoredObjects(
Context & /* context */,
bool /* has_force_restore_data_flag */)
{
DatabaseOnDisk::iterateTableFiles(*this, log, [this](const String & file_name)
DatabaseOnDisk::iterateMetadataFiles(*this, log, [this](const String & file_name)
{
const std::string table_name = file_name.substr(0, file_name.size() - 4);
attachTable(table_name, nullptr);
@ -70,7 +68,16 @@ void DatabaseLazy::createTable(
std::lock_guard lock(tables_mutex);
auto it = tables_cache.find(table_name);
if (it != tables_cache.end())
it->second.metadata_modification_time = DatabaseOnDisk::getTableMetadataModificationTime(*this, table_name);
it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name);
}
void DatabaseLazy::createDictionary(
const Context & /*context*/,
const String & /*dictionary_name*/,
const ASTPtr & /*query*/)
{
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
}
@ -82,6 +89,51 @@ void DatabaseLazy::removeTable(
DatabaseOnDisk::removeTable(*this, context, table_name, log);
}
void DatabaseLazy::removeDictionary(
const Context & /*context*/,
const String & /*table_name*/)
{
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
}
ASTPtr DatabaseLazy::getCreateDictionaryQuery(
const Context & /*context*/,
const String & /*table_name*/) const
{
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
}
ASTPtr DatabaseLazy::tryGetCreateDictionaryQuery(const Context & /*context*/, const String & /*table_name*/) const
{
return nullptr;
}
bool DatabaseLazy::isDictionaryExist(const Context & /*context*/, const String & /*table_name*/) const
{
return false;
}
DatabaseDictionariesIteratorPtr DatabaseLazy::getDictionariesIterator(
const Context & /*context*/,
const FilterByNameFunction & /*filter_by_dictionary_name*/)
{
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
}
void DatabaseLazy::attachDictionary(
const String & /*dictionary_name*/,
const Context & /*context*/,
bool /*load*/)
{
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
{
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseLazy::renameTable(
const Context & context,
const String & table_name,
@ -94,7 +146,7 @@ void DatabaseLazy::renameTable(
}
time_t DatabaseLazy::getTableMetadataModificationTime(
time_t DatabaseLazy::getObjectMetadataModificationTime(
const Context & /* context */,
const String & table_name)
{
@ -172,7 +224,7 @@ StoragePtr DatabaseLazy::tryGetTable(
return loadTable(context, table_name);
}
DatabaseIteratorPtr DatabaseLazy::getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name)
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name)
{
std::lock_guard lock(tables_mutex);
Strings filtered_tables;
@ -198,7 +250,7 @@ void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & tab
auto [it, inserted] = tables_cache.emplace(std::piecewise_construct,
std::forward_as_tuple(table_name),
std::forward_as_tuple(table, current_time, DatabaseOnDisk::getTableMetadataModificationTime(*this, table_name)));
std::forward_as_tuple(table, current_time, DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name)));
if (!inserted)
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
@ -267,9 +319,9 @@ String DatabaseLazy::getDatabaseName() const
return name;
}
String DatabaseLazy::getTableMetadataPath(const String & table_name) const
String DatabaseLazy::getObjectMetadataPath(const String & table_name) const
{
return DatabaseOnDisk::getTableMetadataPath(*this, table_name);
return DatabaseOnDisk::getObjectMetadataPath(*this, table_name);
}
StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table_name) const
@ -280,31 +332,18 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql";
String s;
{
char in_buf[METADATA_FILE_BUFFER_SIZE];
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
readStringUntilEOF(s, in);
}
/** Empty files with metadata are generated after a rough restart of the server.
* Remove these files to slightly reduce the work of the admins on startup.
*/
if (s.empty())
{
LOG_ERROR(log, "LoadTable: File " << table_metadata_path << " is empty. Removing.");
Poco::File(table_metadata_path).remove();
return nullptr;
}
try
{
String table_name_;
StoragePtr table;
Context context_copy(context); /// some tables can change context, but not LogTables
std::tie(table_name_, table) = createTableFromDefinition(
s, name, getDataPath(), context_copy, false, "in file " + table_metadata_path);
if (!endsWith(table->getName(), "Log"))
auto ast = parseCreateQueryFromMetadataFile(table_metadata_path, log);
if (ast)
std::tie(table_name_, table) = createTableFromAST(
ast->as<const ASTCreateQuery &>(), name, getDataPath(), context_copy, false);
if (!ast || !endsWith(table->getName(), "Log"))
throw Exception("Only *Log tables can be used with Lazy database engine.", ErrorCodes::LOGICAL_ERROR);
{
std::lock_guard lock(tables_mutex);

View File

@ -2,11 +2,13 @@
#include <Databases/DatabasesCommon.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>
namespace DB
{
class DatabaseLazyIterator;
/** Lazy engine of databases.
@ -20,7 +22,7 @@ public:
String getEngineName() const override { return "Lazy"; }
void loadTables(
void loadStoredObjects(
Context & context,
bool has_force_restore_data_flag) override;
@ -30,10 +32,19 @@ public:
const StoragePtr & table,
const ASTPtr & query) override;
void createDictionary(
const Context & context,
const String & dictionary_name,
const ASTPtr & query) override;
void removeTable(
const Context & context,
const String & table_name) override;
void removeDictionary(
const Context & context,
const String & table_name) override;
void renameTable(
const Context & context,
const String & table_name,
@ -49,7 +60,7 @@ public:
const ConstraintsDescription & constraints,
const ASTModifier & engine_modifier) override;
time_t getTableMetadataModificationTime(
time_t getObjectMetadataModificationTime(
const Context & context,
const String & table_name) override;
@ -61,12 +72,20 @@ public:
const Context & context,
const String & table_name) const override;
ASTPtr getCreateDictionaryQuery(
const Context & context,
const String & dictionary_name) const override;
ASTPtr tryGetCreateDictionaryQuery(
const Context & context,
const String & dictionary_name) const override;
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
String getDataPath() const override;
String getDatabaseName() const override;
String getMetadataPath() const override;
String getTableMetadataPath(const String & table_name) const override;
String getObjectMetadataPath(const String & table_name) const override;
void drop() override;
@ -74,18 +93,28 @@ public:
const Context & context,
const String & table_name) const override;
bool isDictionaryExist(
const Context & context,
const String & table_name) const override;
StoragePtr tryGetTable(
const Context & context,
const String & table_name) const override;
bool empty(const Context & context) const override;
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
void attachTable(const String & table_name, const StoragePtr & table) override;
StoragePtr detachTable(const String & table_name) override;
void attachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void detachDictionary(const String & dictionary_name, const Context & context, bool reload) override;
void shutdown() override;
~DatabaseLazy() override;
@ -138,10 +167,13 @@ private:
};
class DatabaseLazyIterator final : public IDatabaseIterator
class DatabaseLazyIterator final : public IDatabaseTablesIterator
{
public:
DatabaseLazyIterator(DatabaseLazy & database_, const Context & context_, Strings && table_names_);
DatabaseLazyIterator(
DatabaseLazy & database_,
const Context & context_,
Strings && table_names_);
void next() override;
bool isValid() const override;
@ -155,5 +187,4 @@ private:
Strings::const_iterator iterator;
mutable StoragePtr current_storage;
};
}

View File

@ -9,6 +9,8 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY;
extern const int UNSUPPORTED_METHOD;
}
DatabaseMemory::DatabaseMemory(String name_)
@ -16,7 +18,7 @@ DatabaseMemory::DatabaseMemory(String name_)
, log(&Logger::get("DatabaseMemory(" + name + ")"))
{}
void DatabaseMemory::loadTables(
void DatabaseMemory::loadStoredObjects(
Context & /*context*/,
bool /*has_force_restore_data_flag*/)
{
@ -32,6 +34,21 @@ void DatabaseMemory::createTable(
attachTable(table_name, table);
}
void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
{
throw Exception("There is no ATTACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseMemory::createDictionary(
const Context & /*context*/,
const String & /*dictionary_name*/,
const ASTPtr & /*query*/)
{
throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseMemory::removeTable(
const Context & /*context*/,
const String & table_name)
@ -39,9 +56,23 @@ void DatabaseMemory::removeTable(
detachTable(table_name);
}
time_t DatabaseMemory::getTableMetadataModificationTime(
const Context &,
const String &)
void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
{
throw Exception("There is no DETACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
}
void DatabaseMemory::removeDictionary(
const Context & /*context*/,
const String & /*dictionary_name*/)
{
throw Exception("There is no DROP DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
}
time_t DatabaseMemory::getObjectMetadataModificationTime(
const Context &, const String &)
{
return static_cast<time_t>(0);
}
@ -53,6 +84,15 @@ ASTPtr DatabaseMemory::getCreateTableQuery(
throw Exception("There is no CREATE TABLE query for DatabaseMemory tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
}
ASTPtr DatabaseMemory::getCreateDictionaryQuery(
const Context &,
const String &) const
{
throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory dictionaries", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY);
}
ASTPtr DatabaseMemory::getCreateDatabaseQuery(
const Context &) const
{

View File

@ -23,7 +23,7 @@ public:
String getEngineName() const override { return "Memory"; }
void loadTables(
void loadStoredObjects(
Context & context,
bool has_force_restore_data_flag) override;
@ -33,16 +33,35 @@ public:
const StoragePtr & table,
const ASTPtr & query) override;
void createDictionary(
const Context & context,
const String & dictionary_name,
const ASTPtr & query) override;
void attachDictionary(
const String & name,
const Context & context,
bool reload) override;
void removeTable(
const Context & context,
const String & table_name) override;
time_t getTableMetadataModificationTime(
void removeDictionary(
const Context & context,
const String & table_name) override;
const String & dictionary_name) override;
void detachDictionary(
const String & name,
const Context & context,
bool reload) override;
time_t getObjectMetadataModificationTime(const Context & context, const String & table_name) override;
ASTPtr getCreateTableQuery(const Context & context, const String & table_name) const override;
ASTPtr getCreateDictionaryQuery(const Context & context, const String & table_name) const override;
ASTPtr tryGetCreateTableQuery(const Context &, const String &) const override { return nullptr; }
ASTPtr tryGetCreateDictionaryQuery(const Context &, const String &) const override { return nullptr; }
ASTPtr getCreateDatabaseQuery(const Context & context) const override;

View File

@ -64,7 +64,7 @@ bool DatabaseMySQL::empty(const Context &) const
return local_tables_cache.empty();
}
DatabaseIteratorPtr DatabaseMySQL::getIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
{
Tables tables;
std::lock_guard<std::mutex> lock(mutex);
@ -75,7 +75,7 @@ DatabaseIteratorPtr DatabaseMySQL::getIterator(const Context &, const FilterByNa
if (!filter_by_table_name || filter_by_table_name(local_table.first))
tables[local_table.first] = local_table.second.storage;
return std::make_unique<DatabaseSnapshotIterator>(tables);
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
}
bool DatabaseMySQL::isTableExist(const Context & context, const String & name) const
@ -107,7 +107,7 @@ ASTPtr DatabaseMySQL::tryGetCreateTableQuery(const Context &, const String & tab
return local_tables_cache[table_name].create_table_query;
}
time_t DatabaseMySQL::getTableMetadataModificationTime(const Context &, const String & table_name)
time_t DatabaseMySQL::getObjectMetadataModificationTime(const Context &, const String & table_name)
{
std::lock_guard<std::mutex> lock(mutex);

View File

@ -6,6 +6,7 @@
#include <mysqlxx/Pool.h>
#include <Databases/DatabasesCommon.h>
#include <Interpreters/Context.h>
#include <memory>
namespace DB
@ -29,17 +30,32 @@ public:
bool empty(const Context & context) const override;
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context &, const FilterByNameFunction & = {}) override
{
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
}
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
bool isTableExist(const Context & context, const String & name) const override;
bool isDictionaryExist(const Context &, const String &) const override { return false; }
StoragePtr tryGetTable(const Context & context, const String & name) const override;
ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const override;
time_t getTableMetadataModificationTime(const Context & context, const String & name) override;
ASTPtr getCreateDictionaryQuery(const Context &, const String &) const override
{
throw Exception("MySQL database engine does not support dictionaries.", ErrorCodes::NOT_IMPLEMENTED);
}
ASTPtr tryGetCreateDictionaryQuery(const Context &, const String &) const override { return nullptr; }
time_t getObjectMetadataModificationTime(const Context & context, const String & name) override;
void shutdown() override;
@ -48,7 +64,12 @@ public:
throw Exception("MySQL database engine does not support detach table.", ErrorCodes::NOT_IMPLEMENTED);
}
void loadTables(Context &, bool) override
void detachDictionary(const String &, const Context &, bool) override
{
throw Exception("MySQL database engine does not support detach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
void loadStoredObjects(Context &, bool) override
{
/// do nothing
}
@ -58,16 +79,33 @@ public:
throw Exception("MySQL database engine does not support remove table.", ErrorCodes::NOT_IMPLEMENTED);
}
void removeDictionary(const Context &, const String &) override
{
throw Exception("MySQL database engine does not support remove dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
void attachTable(const String &, const StoragePtr &) override
{
throw Exception("MySQL database engine does not support attach table.", ErrorCodes::NOT_IMPLEMENTED);
}
void attachDictionary(const String &, const Context &, bool) override
{
throw Exception("MySQL database engine does not support attach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
void createTable(const Context &, const String &, const StoragePtr &, const ASTPtr &) override
{
throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED);
}
void createDictionary(const Context &, const String &, const ASTPtr &) override
{
throw Exception("MySQL database engine does not support create dictionary.", ErrorCodes::NOT_IMPLEMENTED);
}
private:
struct MySQLStorageInfo
{

View File

@ -1,37 +1,49 @@
#include <Databases/DatabaseOnDisk.h>
#include <Common/escapeForFileName.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Storages/IStorage.h>
#include <Storages/StorageFactory.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Common/escapeForFileName.h>
#include <common/logger_useful.h>
#include <Poco/DirectoryIterator.h>
namespace DB
{
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
namespace ErrorCodes
{
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY;
extern const int FILE_DOESNT_EXIST;
extern const int INCORRECT_FILE_NAME;
extern const int SYNTAX_ERROR;
extern const int TABLE_ALREADY_EXISTS;
extern const int UNKNOWN_TABLE;
extern const int DICTIONARY_ALREADY_EXISTS;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
}
namespace detail
{
String getTableMetadataPath(const String & base_path, const String & table_name)
String getObjectMetadataPath(const String & base_path, const String & table_name)
{
return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql";
}
@ -85,6 +97,107 @@ namespace detail
}
}
ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log)
{
String definition;
{
char in_buf[METADATA_FILE_BUFFER_SIZE];
ReadBufferFromFile in(filepath, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
readStringUntilEOF(definition, in);
}
/** Empty files with metadata are generated after a rough restart of the server.
* Remove these files to slightly reduce the work of the admins on startup.
*/
if (definition.empty())
{
LOG_ERROR(log, "File " << filepath << " is empty. Removing.");
Poco::File(filepath).remove();
return nullptr;
}
ParserCreateQuery parser_create;
ASTPtr result = parseQuery(parser_create, definition, "in file " + filepath, 0);
return result;
}
std::pair<String, StoragePtr> createTableFromAST(
ASTCreateQuery ast_create_query,
const String & database_name,
const String & database_data_path,
Context & context,
bool has_force_restore_data_flag)
{
ast_create_query.attach = true;
ast_create_query.database = database_name;
if (ast_create_query.as_table_function)
{
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
const auto & factory = TableFunctionFactory::instance();
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
return {ast_create_query.table, storage};
}
/// We do not directly use `InterpreterCreateQuery::execute`, because
/// - the database has not been created yet;
/// - the code is simpler, since the query is already brought to a suitable form.
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context);
ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
return
{
ast_create_query.table,
StorageFactory::instance().get(
ast_create_query,
database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(),
columns, constraints,
true, has_force_restore_data_flag)
};
}
String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
{
ASTPtr query_clone = query->clone();
auto * create = query_clone->as<ASTCreateQuery>();
if (!create)
{
std::ostringstream query_stream;
formatAST(*create, query_stream, true);
throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR);
}
if (!create->is_dictionary)
create->attach = true;
/// We remove everything that is not needed for ATTACH from the query.
create->database.clear();
create->as_database.clear();
create->as_table.clear();
create->if_not_exists = false;
create->is_populate = false;
create->replace_view = false;
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
if (!create->is_view && !create->is_materialized_view && !create->is_live_view)
create->select = nullptr;
create->format = nullptr;
create->out_file = nullptr;
std::ostringstream statement_stream;
formatAST(*create, statement_stream, false);
statement_stream << '\n';
return statement_stream.str();
}
void DatabaseOnDisk::createTable(
IDatabase & database,
const Context & context,
@ -106,15 +219,19 @@ void DatabaseOnDisk::createTable(
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
if (database.isDictionaryExist(context, table_name))
throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " already exists.",
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
if (database.isTableExist(context, table_name))
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
String table_metadata_path = database.getTableMetadataPath(table_name);
String table_metadata_path = database.getObjectMetadataPath(table_name);
String table_metadata_tmp_path = table_metadata_path + ".tmp";
String statement;
{
statement = getTableDefinitionFromCreateQuery(query);
statement = getObjectDefinitionFromCreateQuery(query);
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
@ -141,6 +258,70 @@ void DatabaseOnDisk::createTable(
}
}
void DatabaseOnDisk::createDictionary(
IDatabase & database,
const Context & context,
const String & dictionary_name,
const ASTPtr & query)
{
const auto & settings = context.getSettingsRef();
/** The code is based on the assumption that all threads share the same order of operations
* - creating the .sql.tmp file;
* - adding a dictionary to `dictionaries`;
* - rename .sql.tmp to .sql.
*/
/// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH.
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
if (database.isDictionaryExist(context, dictionary_name))
throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
if (database.isTableExist(context, dictionary_name))
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name);
String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp";
String statement;
{
statement = getObjectDefinitionFromCreateQuery(query);
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
WriteBufferFromFile out(dictionary_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
writeString(statement, out);
out.next();
if (settings.fsync_metadata)
out.sync();
out.close();
}
try
{
/// Do not load it now because we want more strict loading
database.attachDictionary(dictionary_name, context, false);
/// Load dictionary
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
String dict_name = database.getDatabaseName() + "." + dictionary_name;
context.getExternalDictionariesLoader().addDictionaryWithConfig(
dict_name, database.getDatabaseName(), query->as<const ASTCreateQuery &>(), !lazy_load);
/// If it was ATTACH query and file with dictionary metadata already exist
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path);
}
catch (...)
{
database.detachDictionary(dictionary_name, context);
Poco::File(dictionary_metadata_tmp_path).remove();
throw;
}
}
void DatabaseOnDisk::removeTable(
IDatabase & database,
const Context & /* context */,
@ -149,7 +330,7 @@ void DatabaseOnDisk::removeTable(
{
StoragePtr res = database.detachTable(table_name);
String table_metadata_path = database.getTableMetadataPath(table_name);
String table_metadata_path = database.getObjectMetadataPath(table_name);
try
{
@ -171,12 +352,39 @@ void DatabaseOnDisk::removeTable(
}
}
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const IDatabase & database, const Context & context,
const String & table_name, bool throw_on_error)
void DatabaseOnDisk::removeDictionary(
IDatabase & database,
const Context & context,
const String & dictionary_name,
Poco::Logger * /*log*/)
{
database.detachDictionary(dictionary_name, context);
String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name);
try
{
Poco::File(dictionary_metadata_path).remove();
}
catch (...)
{
/// If remove was not possible for some reason
database.attachDictionary(dictionary_name, context);
throw;
}
}
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(
const IDatabase & database,
const Context & context,
const String & table_name,
bool throw_on_error)
{
ASTPtr ast;
auto table_metadata_path = detail::getTableMetadataPath(database.getMetadataPath(), table_name);
auto table_metadata_path = detail::getObjectMetadataPath(database.getMetadataPath(), table_name);
ast = detail::getCreateQueryFromMetadata(table_metadata_path, database.getDatabaseName(), throw_on_error);
if (!ast && throw_on_error)
{
@ -193,6 +401,30 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const IDatabase & database, const
return ast;
}
ASTPtr DatabaseOnDisk::getCreateDictionaryQueryImpl(
const IDatabase & database,
const Context & context,
const String & dictionary_name,
bool throw_on_error)
{
ASTPtr ast;
auto dictionary_metadata_path = detail::getObjectMetadataPath(database.getMetadataPath(), dictionary_name);
ast = detail::getCreateQueryFromMetadata(dictionary_metadata_path, database.getDatabaseName(), throw_on_error);
if (!ast && throw_on_error)
{
/// Handle system.* tables for which there are no table.sql files.
bool has_dictionary = database.isDictionaryExist(context, dictionary_name);
auto msg = has_dictionary ? "There is no CREATE DICTIONARY query for table " : "There is no metadata file for dictionary ";
throw Exception(msg + backQuote(dictionary_name), ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY);
}
return ast;
}
ASTPtr DatabaseOnDisk::getCreateTableQuery(const IDatabase & database, const Context & context, const String & table_name)
{
return getCreateTableQueryImpl(database, context, table_name, true);
@ -203,6 +435,17 @@ ASTPtr DatabaseOnDisk::tryGetCreateTableQuery(const IDatabase & database, const
return getCreateTableQueryImpl(database, context, table_name, false);
}
ASTPtr DatabaseOnDisk::getCreateDictionaryQuery(const IDatabase & database, const Context & context, const String & dictionary_name)
{
return getCreateDictionaryQueryImpl(database, context, dictionary_name, true);
}
ASTPtr DatabaseOnDisk::tryGetCreateDictionaryQuery(const IDatabase & database, const Context & context, const String & dictionary_name)
{
return getCreateDictionaryQueryImpl(database, context, dictionary_name, false);
}
ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const Context & /*context*/)
{
ASTPtr ast;
@ -226,29 +469,25 @@ void DatabaseOnDisk::drop(const IDatabase & database)
Poco::File(database.getMetadataPath()).remove(false);
}
String DatabaseOnDisk::getTableMetadataPath(const IDatabase & database, const String & table_name)
String DatabaseOnDisk::getObjectMetadataPath(const IDatabase & database, const String & table_name)
{
return detail::getTableMetadataPath(database.getMetadataPath(), table_name);
return detail::getObjectMetadataPath(database.getMetadataPath(), table_name);
}
time_t DatabaseOnDisk::getTableMetadataModificationTime(
time_t DatabaseOnDisk::getObjectMetadataModificationTime(
const IDatabase & database,
const String & table_name)
{
String table_metadata_path = getTableMetadataPath(database, table_name);
String table_metadata_path = getObjectMetadataPath(database, table_name);
Poco::File meta_file(table_metadata_path);
if (meta_file.exists())
{
return meta_file.getLastModified().epochTime();
}
else
{
return static_cast<time_t>(0);
}
}
void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function)
void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function)
{
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it)
@ -265,11 +504,11 @@ void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger
static const char * tmp_drop_ext = ".sql.tmp_drop";
if (endsWith(dir_it.name(), tmp_drop_ext))
{
const std::string table_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
if (Poco::File(database.getDataPath() + '/' + table_name).exists())
const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
if (Poco::File(database.getDataPath() + '/' + object_name).exists())
{
Poco::File(dir_it->path()).renameTo(table_name + ".sql");
LOG_WARNING(log, "Table " << backQuote(table_name) << " was not dropped previously");
Poco::File(dir_it->path()).renameTo(object_name + ".sql");
LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously");
}
else
{

View File

@ -13,12 +13,28 @@ namespace DB
namespace detail
{
String getTableMetadataPath(const String & base_path, const String & table_name);
String getObjectMetadataPath(const String & base_path, const String & dictionary_name);
String getDatabaseMetadataPath(const String & base_path);
ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true);
ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error);
}
ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log);
std::pair<String, StoragePtr> createTableFromAST(
ASTCreateQuery ast_create_query,
const String & database_name,
const String & database_data_path,
Context & context,
bool has_force_restore_data_flag);
/** Get the row with the table definition based on the CREATE query.
* It is an ATTACH query that you can execute to create a table from the correspondent database.
* See the implementation.
*/
String getObjectDefinitionFromCreateQuery(const ASTPtr & query);
/* Class to provide basic operations with tables when metadata is stored on disk in .sql files.
*/
class DatabaseOnDisk
@ -31,12 +47,24 @@ public:
const StoragePtr & table,
const ASTPtr & query);
static void createDictionary(
IDatabase & database,
const Context & context,
const String & dictionary_name,
const ASTPtr & query);
static void removeTable(
IDatabase & database,
const Context & context,
const String & table_name,
Poco::Logger * log);
static void removeDictionary(
IDatabase & database,
const Context & context,
const String & dictionary_name,
Poco::Logger * log);
template <typename Database>
static void renameTable(
IDatabase & database,
@ -56,23 +84,33 @@ public:
const Context & context,
const String & table_name);
static ASTPtr getCreateDictionaryQuery(
const IDatabase & database,
const Context & context,
const String & dictionary_name);
static ASTPtr tryGetCreateDictionaryQuery(
const IDatabase & database,
const Context & context,
const String & dictionary_name);
static ASTPtr getCreateDatabaseQuery(
const IDatabase & database,
const Context & context);
static void drop(const IDatabase & database);
static String getTableMetadataPath(
static String getObjectMetadataPath(
const IDatabase & database,
const String & table_name);
const String & object_name);
static time_t getTableMetadataModificationTime(
static time_t getObjectMetadataModificationTime(
const IDatabase & database,
const String & table_name);
const String & object_name);
using IteratingFunction = std::function<void(const String &)>;
static void iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function);
static void iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function);
private:
static ASTPtr getCreateTableQueryImpl(
@ -80,6 +118,12 @@ private:
const Context & context,
const String & table_name,
bool throw_on_error);
static ASTPtr getCreateDictionaryQueryImpl(
const IDatabase & database,
const Context & context,
const String & dictionary_name,
bool throw_on_error);
};
@ -126,7 +170,7 @@ void DatabaseOnDisk::renameTable(
throw Exception{Exception::CreateFromPoco, e};
}
ASTPtr ast = detail::getQueryFromMetadata(detail::getTableMetadataPath(database.getMetadataPath(), table_name));
ASTPtr ast = detail::getQueryFromMetadata(detail::getObjectMetadataPath(database.getMetadataPath(), table_name));
if (!ast)
throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST);
ast->as<ASTCreateQuery &>().table = to_table_name;

View File

@ -11,10 +11,17 @@
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserDictionary.h>
#include <Storages/StorageFactory.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <Storages/IStorage.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Poco/DirectoryIterator.h>
#include <Poco/Event.h>
@ -33,59 +40,64 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_CREATE_TABLE_FROM_METADATA;
extern const int CANNOT_CREATE_DICTIONARY_FROM_METADATA;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
extern const int CANNOT_PARSE_TEXT;
extern const int EMPTY_LIST_OF_ATTRIBUTES_PASSED;
}
static constexpr size_t PRINT_MESSAGE_EACH_N_TABLES = 256;
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
static void loadTable(
Context & context,
const String & database_metadata_path,
DatabaseOrdinary & database,
const String & database_name,
const String & database_data_path,
const String & file_name,
bool has_force_restore_data_flag)
namespace
{
Logger * log = &Logger::get("loadTable");
const String table_metadata_path = database_metadata_path + "/" + file_name;
String s;
void loadObject(
Context & context,
const ASTCreateQuery & query,
DatabaseOrdinary & database,
const String database_data_path,
const String & database_name,
bool has_force_restore_data_flag)
try
{
if (query.is_dictionary)
{
char in_buf[METADATA_FILE_BUFFER_SIZE];
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
readStringUntilEOF(s, in);
String dictionary_name = query.table;
database.attachDictionary(dictionary_name, context, false);
}
/** Empty files with metadata are generated after a rough restart of the server.
* Remove these files to slightly reduce the work of the admins on startup.
*/
if (s.empty())
{
LOG_ERROR(log, "File " << table_metadata_path << " is empty. Removing.");
Poco::File(table_metadata_path).remove();
return;
}
try
else
{
String table_name;
StoragePtr table;
std::tie(table_name, table) = createTableFromDefinition(
s, database_name, database_data_path, context, has_force_restore_data_flag, "in file " + table_metadata_path);
std::tie(table_name, table)
= createTableFromAST(query, database_name, database_data_path, context, has_force_restore_data_flag);
database.attachTable(table_name, table);
}
catch (const Exception & e)
}
catch (const Exception & e)
{
throw Exception(
"Cannot create object '" + query.table + "' from query " + serializeAST(query) + ", error: " + e.displayText() + ", stack trace:\n"
+ e.getStackTrace().toString(),
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
}
void logAboutProgress(Poco::Logger * log, size_t processed, size_t total, AtomicStopwatch & watch)
{
if (processed % PRINT_MESSAGE_EACH_N_OBJECTS == 0 || watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
{
throw Exception("Cannot create table from metadata file " + table_metadata_path + ", error: " + e.displayText() +
", stack trace:\n" + e.getStackTrace().toString(),
ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA);
LOG_INFO(log, std::fixed << std::setprecision(2) << processed * 100.0 / total << "%");
watch.restart();
}
}
}
DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context)
: DatabaseWithOwnTablesBase(std::move(name_))
@ -97,57 +109,78 @@ DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_,
}
void DatabaseOrdinary::loadTables(
void DatabaseOrdinary::loadStoredObjects(
Context & context,
bool has_force_restore_data_flag)
{
using FileNames = std::vector<std::string>;
FileNames file_names;
DatabaseOnDisk::iterateTableFiles(*this, log, [&file_names](const String & file_name)
{
file_names.push_back(file_name);
});
if (file_names.empty())
return;
/** Tables load faster if they are loaded in sorted (by name) order.
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
* which does not correspond to order tables creation and does not correspond to order of their location on disk.
*/
std::sort(file_names.begin(), file_names.end());
using FileNames = std::map<std::string, ASTPtr>;
FileNames file_names;
const size_t total_tables = file_names.size();
LOG_INFO(log, "Total " << total_tables << " tables.");
size_t total_dictionaries = 0;
DatabaseOnDisk::iterateMetadataFiles(*this, log, [&file_names, &total_dictionaries, this](const String & file_name)
{
String full_path = metadata_path + "/" + file_name;
try
{
auto ast = parseCreateQueryFromMetadataFile(full_path, log);
if (ast)
{
auto * create_query = ast->as<ASTCreateQuery>();
file_names[file_name] = ast;
total_dictionaries += create_query->is_dictionary;
}
}
catch (const Exception & e)
{
throw Exception(
"Cannot parse definition from metadata file " + full_path + ", error: " + e.displayText() + ", stack trace:\n"
+ e.getStackTrace().toString(), ErrorCodes::CANNOT_PARSE_TEXT);
}
});
size_t total_tables = file_names.size() - total_dictionaries;
LOG_INFO(log, "Total " << total_tables << " tables and " << total_dictionaries << " dictionaries.");
AtomicStopwatch watch;
std::atomic<size_t> tables_processed {0};
std::atomic<size_t> tables_processed{0};
std::atomic<size_t> dictionaries_processed{0};
auto loadOneTable = [&](const String & table)
auto loadOneObject = [&](const ASTCreateQuery & query)
{
loadTable(context, getMetadataPath(), *this, getDatabaseName(), getDataPath(), table, has_force_restore_data_flag);
loadObject(context, query, *this, getDataPath(), getDatabaseName(), has_force_restore_data_flag);
/// Messages, so that it's not boring to wait for the server to load for a long time.
if (++tables_processed % PRINT_MESSAGE_EACH_N_TABLES == 0
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
{
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
watch.restart();
}
if (query.is_dictionary)
logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch);
else
logAboutProgress(log, ++tables_processed, total_tables, watch);
};
ThreadPool pool(SettingMaxThreads().getAutoValue());
for (const auto & file_name : file_names)
for (const auto & name_with_query : file_names)
{
pool.scheduleOrThrowOnError([&]() { loadOneTable(file_name); });
pool.scheduleOrThrowOnError([&]() { loadOneObject(name_with_query.second->as<const ASTCreateQuery &>()); });
}
pool.wait();
/// After all tables was basically initialized, startup them.
startupTables(pool);
/// Add database as repository
auto dictionaries_repository = std::make_unique<ExternalLoaderDatabaseConfigRepository>(shared_from_this(), context);
auto & external_loader = context.getExternalDictionariesLoader();
external_loader.addConfigRepository(getDatabaseName(), std::move(dictionaries_repository));
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
external_loader.reload(!lazy_load);
}
@ -160,18 +193,12 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
return;
AtomicStopwatch watch;
std::atomic<size_t> tables_processed {0};
std::atomic<size_t> tables_processed{0};
auto startupOneTable = [&](const StoragePtr & table)
{
table->startup();
if (++tables_processed % PRINT_MESSAGE_EACH_N_TABLES == 0
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
{
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
watch.restart();
}
logAboutProgress(log, ++tables_processed, total_tables, watch);
};
try
@ -187,7 +214,6 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
thread_pool.wait();
}
void DatabaseOrdinary::createTable(
const Context & context,
const String & table_name,
@ -197,6 +223,13 @@ void DatabaseOrdinary::createTable(
DatabaseOnDisk::createTable(*this, context, table_name, table, query);
}
void DatabaseOrdinary::createDictionary(
const Context & context,
const String & dictionary_name,
const ASTPtr & query)
{
DatabaseOnDisk::createDictionary(*this, context, dictionary_name, query);
}
void DatabaseOrdinary::removeTable(
const Context & context,
@ -205,6 +238,13 @@ void DatabaseOrdinary::removeTable(
DatabaseOnDisk::removeTable(*this, context, table_name, log);
}
void DatabaseOrdinary::removeDictionary(
const Context & context,
const String & table_name)
{
DatabaseOnDisk::removeDictionary(*this, context, table_name, log);
}
void DatabaseOrdinary::renameTable(
const Context & context,
const String & table_name,
@ -216,11 +256,11 @@ void DatabaseOrdinary::renameTable(
}
time_t DatabaseOrdinary::getTableMetadataModificationTime(
time_t DatabaseOrdinary::getObjectMetadataModificationTime(
const Context & /* context */,
const String & table_name)
{
return DatabaseOnDisk::getTableMetadataModificationTime(*this, table_name);
return DatabaseOnDisk::getObjectMetadataModificationTime(*this, table_name);
}
ASTPtr DatabaseOrdinary::getCreateTableQuery(const Context & context, const String & table_name) const
@ -233,6 +273,17 @@ ASTPtr DatabaseOrdinary::tryGetCreateTableQuery(const Context & context, const S
return DatabaseOnDisk::tryGetCreateTableQuery(*this, context, table_name);
}
ASTPtr DatabaseOrdinary::getCreateDictionaryQuery(const Context & context, const String & dictionary_name) const
{
return DatabaseOnDisk::getCreateDictionaryQuery(*this, context, dictionary_name);
}
ASTPtr DatabaseOrdinary::tryGetCreateDictionaryQuery(const Context & context, const String & dictionary_name) const
{
return DatabaseOnDisk::tryGetCreateTableQuery(*this, context, dictionary_name);
}
ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & context) const
{
return DatabaseOnDisk::getCreateDatabaseQuery(*this, context);
@ -283,7 +334,7 @@ void DatabaseOrdinary::alterTable(
if (storage_modifier)
storage_modifier(*ast_create_query.storage);
statement = getTableDefinitionFromCreateQuery(ast);
statement = getObjectDefinitionFromCreateQuery(ast);
{
WriteBufferFromFile out(table_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
@ -328,9 +379,9 @@ String DatabaseOrdinary::getDatabaseName() const
return name;
}
String DatabaseOrdinary::getTableMetadataPath(const String & table_name) const
String DatabaseOrdinary::getObjectMetadataPath(const String & table_name) const
{
return detail::getTableMetadataPath(getMetadataPath(), table_name);
return DatabaseOnDisk::getObjectMetadataPath(*this, table_name);
}
}

View File

@ -18,7 +18,7 @@ public:
String getEngineName() const override { return "Ordinary"; }
void loadTables(
void loadStoredObjects(
Context & context,
bool has_force_restore_data_flag) override;
@ -28,10 +28,19 @@ public:
const StoragePtr & table,
const ASTPtr & query) override;
void createDictionary(
const Context & context,
const String & dictionary_name,
const ASTPtr & query) override;
void removeTable(
const Context & context,
const String & table_name) override;
void removeDictionary(
const Context & context,
const String & table_name) override;
void renameTable(
const Context & context,
const String & table_name,
@ -47,7 +56,7 @@ public:
const ConstraintsDescription & constraints,
const ASTModifier & engine_modifier) override;
time_t getTableMetadataModificationTime(
time_t getObjectMetadataModificationTime(
const Context & context,
const String & table_name) override;
@ -59,12 +68,20 @@ public:
const Context & context,
const String & table_name) const override;
ASTPtr tryGetCreateDictionaryQuery(
const Context & context,
const String & name) const override;
ASTPtr getCreateDictionaryQuery(
const Context & context,
const String & name) const override;
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
String getDataPath() const override;
String getDatabaseName() const override;
String getMetadataPath() const override;
String getTableMetadataPath(const String & table_name) const override;
String getObjectMetadataPath(const String & table_name) const override;
void drop() override;
@ -74,8 +91,6 @@ private:
Poco::Logger * log;
void startupTables(ThreadPool & thread_pool);
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
};
}

View File

@ -1,15 +1,20 @@
#include <Databases/DatabasesCommon.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
#include <Interpreters/Context.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserDictionary.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Storages/IStorage.h>
#include <Storages/StorageDictionary.h>
#include <Storages/StorageFactory.h>
#include <Common/typeid_cast.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Dictionaries/DictionaryFactory.h>
#include <sstream>
@ -23,115 +28,119 @@ namespace ErrorCodes
extern const int TABLE_ALREADY_EXISTS;
extern const int UNKNOWN_TABLE;
extern const int LOGICAL_ERROR;
extern const int DICTIONARY_ALREADY_EXISTS;
}
String getTableDefinitionFromCreateQuery(const ASTPtr & query)
namespace
{
ASTPtr query_clone = query->clone();
auto & create = query_clone->as<ASTCreateQuery &>();
/// We remove everything that is not needed for ATTACH from the query.
create.attach = true;
create.database.clear();
create.as_database.clear();
create.as_table.clear();
create.if_not_exists = false;
create.is_populate = false;
create.replace_view = false;
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
if (!create.is_view && !create.is_materialized_view && !create.is_live_view)
create.select = nullptr;
create.format = nullptr;
create.out_file = nullptr;
std::ostringstream statement_stream;
formatAST(create, statement_stream, false);
statement_stream << '\n';
return statement_stream.str();
}
std::pair<String, StoragePtr> createTableFromDefinition(
const String & definition,
const String & database_name,
const String & database_data_path,
Context & context,
bool has_force_restore_data_flag,
const String & description_for_error_message)
StoragePtr getDictionaryStorage(const Context & context, const String & table_name, const String & db_name)
{
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), description_for_error_message, 0);
auto & ast_create_query = ast->as<ASTCreateQuery &>();
ast_create_query.attach = true;
ast_create_query.database = database_name;
if (ast_create_query.as_table_function)
auto dict_name = db_name + "." + table_name;
const auto & external_loader = context.getExternalDictionariesLoader();
auto dict_ptr = external_loader.tryGetDictionary(dict_name);
if (dict_ptr)
{
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
const auto & factory = TableFunctionFactory::instance();
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
return {ast_create_query.table, storage};
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
return StorageDictionary::create(db_name, table_name, ColumnsDescription{columns}, context, true, dict_name);
}
/// We do not directly use `InterpreterCreateQuery::execute`, because
/// - the database has not been created yet;
/// - the code is simpler, since the query is already brought to a suitable form.
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context);
ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
return
{
ast_create_query.table,
StorageFactory::instance().get(
ast_create_query,
database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(),
columns, constraints,
true, has_force_restore_data_flag)
};
return nullptr;
}
}
bool DatabaseWithOwnTablesBase::isTableExist(
const Context & /*context*/,
const String & table_name) const
{
std::lock_guard lock(mutex);
return tables.find(table_name) != tables.end();
return tables.find(table_name) != tables.end() || dictionaries.find(table_name) != dictionaries.end();
}
bool DatabaseWithOwnTablesBase::isDictionaryExist(
const Context & /*context*/,
const String & dictionary_name) const
{
std::lock_guard lock(mutex);
return dictionaries.find(dictionary_name) != dictionaries.end();
}
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
const Context & /*context*/,
const Context & context,
const String & table_name) const
{
std::lock_guard lock(mutex);
auto it = tables.find(table_name);
if (it == tables.end())
return {};
return it->second;
{
std::lock_guard lock(mutex);
auto it = tables.find(table_name);
if (it != tables.end())
return it->second;
}
if (isDictionaryExist(context, table_name))
/// We don't need lock database here, because database doesn't store dictionary itself
/// just metadata
return getDictionaryStorage(context, table_name, getDatabaseName());
return {};
}
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name)
{
auto tables_it = getTablesIterator(context, filter_by_name);
auto dictionaries_it = getDictionariesIterator(context, filter_by_name);
Tables result;
while (tables_it && tables_it->isValid())
{
result.emplace(tables_it->name(), tables_it->table());
tables_it->next();
}
while (dictionaries_it && dictionaries_it->isValid())
{
auto table_name = dictionaries_it->name();
auto table_ptr = getDictionaryStorage(context, table_name, getDatabaseName());
if (table_ptr)
result.emplace(table_name, table_ptr);
dictionaries_it->next();
}
return std::make_unique<DatabaseTablesSnapshotIterator>(result);
}
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
{
std::lock_guard lock(mutex);
if (!filter_by_table_name)
return std::make_unique<DatabaseSnapshotIterator>(tables);
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
Tables filtered_tables;
for (const auto & [table_name, storage] : tables)
if (filter_by_table_name(table_name))
filtered_tables.emplace(table_name, storage);
return std::make_unique<DatabaseSnapshotIterator>(std::move(filtered_tables));
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(filtered_tables));
}
DatabaseDictionariesIteratorPtr DatabaseWithOwnTablesBase::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name)
{
std::lock_guard lock(mutex);
if (!filter_by_dictionary_name)
return std::make_unique<DatabaseDictionariesSnapshotIterator>(dictionaries);
Dictionaries filtered_dictionaries;
for (const auto & dictionary_name : dictionaries)
if (filter_by_dictionary_name(dictionary_name))
filtered_dictionaries.emplace(dictionary_name);
return std::make_unique<DatabaseDictionariesSnapshotIterator>(std::move(filtered_dictionaries));
}
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
{
std::lock_guard lock(mutex);
return tables.empty();
return tables.empty() && dictionaries.empty();
}
StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
@ -139,6 +148,9 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
StoragePtr res;
{
std::lock_guard lock(mutex);
if (dictionaries.count(table_name))
throw Exception("Cannot detach dictionary " + name + "." + table_name + " as table, use DETACH DICTIONARY query.", ErrorCodes::UNKNOWN_TABLE);
auto it = tables.find(table_name);
if (it == tables.end())
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
@ -149,6 +161,21 @@ StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
return res;
}
void DatabaseWithOwnTablesBase::detachDictionary(const String & dictionary_name, const Context & context, bool reload)
{
{
std::lock_guard lock(mutex);
auto it = dictionaries.find(dictionary_name);
if (it == dictionaries.end())
throw Exception("Dictionary " + name + "." + dictionary_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
dictionaries.erase(it);
}
if (reload)
context.getExternalDictionariesLoader().reload(getDatabaseName() + "." + dictionary_name);
}
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
{
std::lock_guard lock(mutex);
@ -156,6 +183,25 @@ void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const Sto
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
void DatabaseWithOwnTablesBase::attachDictionary(const String & dictionary_name, const Context & context, bool load)
{
const auto & external_loader = context.getExternalDictionariesLoader();
String full_name = getDatabaseName() + "." + dictionary_name;
{
std::lock_guard lock(mutex);
auto status = external_loader.getCurrentStatus(full_name);
if (status != ExternalLoader::Status::NOT_EXIST || !dictionaries.emplace(dictionary_name).second)
throw Exception(
"Dictionary " + full_name + " already exists.",
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
}
if (load)
external_loader.reload(full_name, true);
}
void DatabaseWithOwnTablesBase::shutdown()
{
/// You can not hold a lock during shutdown.
@ -174,6 +220,7 @@ void DatabaseWithOwnTablesBase::shutdown()
std::lock_guard lock(mutex);
tables.clear();
dictionaries.clear();
}
DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase()

View File

@ -15,63 +15,6 @@ namespace DB
class Context;
/** Get the row with the table definition based on the CREATE query.
* It is an ATTACH query that you can execute to create a table from the correspondent database.
* See the implementation.
*/
String getTableDefinitionFromCreateQuery(const ASTPtr & query);
/** Create a table by its definition, without using InterpreterCreateQuery.
* (InterpreterCreateQuery has more complex functionality, and it can not be used if the database has not been created yet)
* Returns the table name and the table itself.
* You must subsequently call IStorage::startup method to use the table.
*/
std::pair<String, StoragePtr> createTableFromDefinition(
const String & definition,
const String & database_name,
const String & database_data_path,
Context & context,
bool has_force_restore_data_flag,
const String & description_for_error_message);
/// Copies list of tables and iterates through such snapshot.
class DatabaseSnapshotIterator final : public IDatabaseIterator
{
private:
Tables tables;
Tables::iterator it;
public:
DatabaseSnapshotIterator(Tables & tables_)
: tables(tables_), it(tables.begin()) {}
DatabaseSnapshotIterator(Tables && tables_)
: tables(tables_), it(tables.begin()) {}
void next() override
{
++it;
}
bool isValid() const override
{
return it != tables.end();
}
const String & name() const override
{
return it->first;
}
const StoragePtr & table() const override
{
return it->second;
}
};
/// A base class for databases that manage their own list of tables.
class DatabaseWithOwnTablesBase : public IDatabase
{
@ -80,18 +23,27 @@ public:
const Context & context,
const String & table_name) const override;
bool isDictionaryExist(const Context & context, const String & dictionary_name) const override;
StoragePtr tryGetTable(
const Context & context,
const String & table_name) const override;
bool empty(const Context & context) const override;
void attachTable(const String & table_name, const StoragePtr & table) override;
void attachDictionary(const String & name, const Context & context, bool reload) override;
StoragePtr detachTable(const String & table_name) override;
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
void detachDictionary(const String & name, const Context & context, bool reload) override;
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) override;
void shutdown() override;
@ -102,6 +54,7 @@ protected:
mutable std::mutex mutex;
Tables tables;
Dictionaries dictionaries;
DatabaseWithOwnTablesBase(String name_) : name(std::move(name_)) { }
};

View File

@ -3,6 +3,7 @@
#include <Core/Types.h>
#include <Parsers/IAST_fwd.h>
#include <Storages/IStorage_fwd.h>
#include <Dictionaries/IDictionary.h>
#include <Common/Exception.h>
#include <ctime>
@ -19,16 +20,14 @@ struct ConstraintsDescription;
class ColumnsDescription;
struct IndicesDescription;
struct TableStructureWriteLockHolder;
using Dictionaries = std::set<String>;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
/** Allows to iterate over tables.
*/
class IDatabaseIterator
class IDatabaseTablesIterator
{
public:
virtual void next() = 0;
@ -37,15 +36,57 @@ public:
virtual const String & name() const = 0;
virtual const StoragePtr & table() const = 0;
virtual ~IDatabaseIterator() {}
virtual ~IDatabaseTablesIterator() = default;
};
using DatabaseIteratorPtr = std::unique_ptr<IDatabaseIterator>;
/// Copies list of tables and iterates through such snapshot.
class DatabaseTablesSnapshotIterator : public IDatabaseTablesIterator
{
private:
Tables tables;
Tables::iterator it;
public:
DatabaseTablesSnapshotIterator(Tables & tables_) : tables(tables_), it(tables.begin()) {}
DatabaseTablesSnapshotIterator(Tables && tables_) : tables(tables_), it(tables.begin()) {}
void next() { ++it; }
bool isValid() const { return it != tables.end(); }
const String & name() const { return it->first; }
const StoragePtr & table() const { return it->second; }
};
/// Copies list of dictionaries and iterates through such snapshot.
class DatabaseDictionariesSnapshotIterator
{
private:
Dictionaries dictionaries;
Dictionaries::iterator it;
public:
DatabaseDictionariesSnapshotIterator() = default;
DatabaseDictionariesSnapshotIterator(Dictionaries & dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
DatabaseDictionariesSnapshotIterator(Dictionaries && dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
void next() { ++it; }
bool isValid() const { return !dictionaries.empty() && it != dictionaries.end(); }
const String & name() const { return *it; }
};
using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
using DatabaseDictionariesIteratorPtr = std::unique_ptr<DatabaseDictionariesSnapshotIterator>;
/** Database engine.
* It is responsible for:
* - initialization of set of known tables;
* - initialization of set of known tables and dictionaries;
* - checking existence of a table and getting a table object;
* - retrieving a list of all tables;
* - creating and dropping tables;
@ -60,7 +101,7 @@ public:
/// Load a set of existing tables.
/// You can call only once, right after the object is created.
virtual void loadTables(
virtual void loadStoredObjects(
Context & context,
bool has_force_restore_data_flag) = 0;
@ -69,6 +110,11 @@ public:
const Context & context,
const String & name) const = 0;
/// Check the existence of the dictionary
virtual bool isDictionaryExist(
const Context & context,
const String & name) const = 0;
/// Get the table for work. Return nullptr if there is no table.
virtual StoragePtr tryGetTable(
const Context & context,
@ -78,7 +124,16 @@ public:
/// Get an iterator that allows you to pass through all the tables.
/// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above.
virtual DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
/// Get an iterator to pass through all the dictionaries.
virtual DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) = 0;
/// Get an iterator to pass through all the tables and dictionary tables.
virtual DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name = {})
{
return getTablesIterator(context, filter_by_name);
}
/// Is the database empty.
virtual bool empty(const Context & context) const = 0;
@ -90,17 +145,35 @@ public:
const StoragePtr & table,
const ASTPtr & query) = 0;
/// Add the dictionary to the database. Record its presence in the metadata.
virtual void createDictionary(
const Context & context,
const String & dictionary_name,
const ASTPtr & query) = 0;
/// Delete the table from the database. Delete the metadata.
virtual void removeTable(
const Context & context,
const String & name) = 0;
/// Delete the dictionary from the database. Delete the metadata.
virtual void removeDictionary(
const Context & context,
const String & dictionary_name) = 0;
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
virtual void attachTable(const String & name, const StoragePtr & table) = 0;
/// Add dictionary to the database, but do not add it to the metadata. The database may not support this method.
/// load is false when we starting up and lazy_load is true, so we don't want to load dictionaries synchronously.
virtual void attachDictionary(const String & name, const Context & context, bool reload = true) = 0;
/// Forget about the table without deleting it, and return it. The database may not support this method.
virtual StoragePtr detachTable(const String & name) = 0;
/// Forget about the dictionary without deleting it, and return it. The database may not support this method.
virtual void detachDictionary(const String & name, const Context & context, bool reload = true) = 0;
/// Rename the table and possibly move the table to another database.
virtual void renameTable(
const Context & /*context*/,
@ -128,7 +201,7 @@ public:
}
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.
virtual time_t getTableMetadataModificationTime(
virtual time_t getObjectMetadataModificationTime(
const Context & context,
const String & name) = 0;
@ -140,6 +213,14 @@ public:
return tryGetCreateTableQuery(context, name);
}
/// Get the CREATE DICTIONARY query for the dictionary. Returns nullptr if dictionary doesn't exists.
virtual ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & name) const = 0;
virtual ASTPtr getCreateDictionaryQuery(const Context & context, const String & name) const
{
return tryGetCreateDictionaryQuery(context, name);
}
/// Get the CREATE DATABASE query for current database.
virtual ASTPtr getCreateDatabaseQuery(const Context & context) const = 0;
@ -150,7 +231,7 @@ public:
/// Returns metadata path if the database supports it, empty string otherwise
virtual String getMetadataPath() const { return {}; }
/// Returns metadata path of a concrete table if the database supports it, empty string otherwise
virtual String getTableMetadataPath(const String & /*table_name*/) const { return {}; }
virtual String getObjectMetadataPath(const String & /*table_name*/) const { return {}; }
/// Ask all tables to complete the background threads they are using and delete all table objects.
virtual void shutdown() = 0;

View File

@ -11,8 +11,8 @@ generate_code(CacheDictionary_generate2 UInt8 UInt16 UInt32 UInt64 UInt128 Int8
generate_code(CacheDictionary_generate3 UInt8 UInt16 UInt32 UInt64 UInt128 Int8 Int16 Int32 Int64 Float32 Float64 Decimal32 Decimal64 Decimal128)
add_headers_and_sources(clickhouse_dictionaries ${CMAKE_CURRENT_BINARY_DIR}/generated/)
list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp)
list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h)
list(REMOVE_ITEM clickhouse_dictionaries_sources DictionaryFactory.cpp DictionarySourceFactory.cpp DictionaryStructure.cpp getDictionaryConfigurationFromAST.cpp)
list(REMOVE_ITEM clickhouse_dictionaries_headers DictionaryFactory.h DictionarySourceFactory.h DictionaryStructure.h getDictionaryConfigurationFromAST.h)
add_library(clickhouse_dictionaries ${clickhouse_dictionaries_sources})
target_link_libraries(clickhouse_dictionaries PRIVATE dbms clickhouse_common_io ${BTRIE_LIBRARIES})

View File

@ -611,7 +611,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
};
factory.registerLayout("cache", create_layout);
factory.registerLayout("cache", create_layout, false);
}

View File

@ -52,7 +52,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const Block & sample_block_,
Context & context_)
const Context & context_)
: update_time{std::chrono::system_clock::from_time_t(0)}
, dict_struct{dict_struct_}
, host{config.getString(config_prefix + ".host")}
@ -206,7 +206,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context) -> DictionarySourcePtr
const Context & context) -> DictionarySourcePtr
{
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
};

View File

@ -23,7 +23,7 @@ public:
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const Block & sample_block_,
Context & context);
const Context & context);
/// copy-constructor is provided in order to support cloneability
ClickHouseDictionarySource(const ClickHouseDictionarySource & other);

View File

@ -415,7 +415,7 @@ void registerDictionaryComplexKeyCache(DictionaryFactory & factory)
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
return std::make_unique<ComplexKeyCacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
};
factory.registerLayout("complex_key_cache", create_layout);
factory.registerLayout("complex_key_cache", create_layout, true);
}

View File

@ -755,7 +755,7 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<ComplexKeyHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("complex_key_hashed", create_layout);
factory.registerLayout("complex_key_hashed", create_layout, true);
}

View File

@ -3,6 +3,7 @@
#include <memory>
#include "DictionarySourceFactory.h"
#include "DictionaryStructure.h"
#include "getDictionaryConfigurationFromAST.h"
namespace DB
{
@ -12,15 +13,21 @@ namespace ErrorCodes
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
}
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout)
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex)
{
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
layout_complexity[layout_type] = is_complex;
}
DictionaryPtr DictionaryFactory::create(
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const
const std::string & name,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const Context & context) const
{
Poco::Util::AbstractConfiguration::Keys keys;
const auto & layout_prefix = config_prefix + ".layout";
@ -31,7 +38,7 @@ DictionaryPtr DictionaryFactory::create(
const DictionaryStructure dict_struct{config, config_prefix + ".structure"};
auto source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
const auto & layout_type = keys.front();
@ -39,14 +46,21 @@ DictionaryPtr DictionaryFactory::create(
const auto found = registered_layouts.find(layout_type);
if (found != registered_layouts.end())
{
const auto & create_layout = found->second;
return create_layout(name, dict_struct, config, config_prefix, std::move(source_ptr));
const auto & layout_creator = found->second;
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr));
}
}
throw Exception{name + ": unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
}
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const
{
auto configurationFromAST = getDictionaryConfigurationFromAST(ast);
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context);
}
DictionaryFactory & DictionaryFactory::instance()
{
static DictionaryFactory ret;

View File

@ -1,6 +1,7 @@
#pragma once
#include "IDictionary.h"
#include <Parsers/ASTCreateQuery.h>
namespace Poco
@ -27,7 +28,15 @@ public:
static DictionaryFactory & instance();
DictionaryPtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const;
DictionaryPtr create(
const std::string & name,
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const Context & context) const;
DictionaryPtr create(const std::string & name,
const ASTCreateQuery & ast,
const Context & context) const;
using Creator = std::function<DictionaryPtr(
const std::string & name,
@ -36,11 +45,15 @@ public:
const std::string & config_prefix,
DictionarySourcePtr source_ptr)>;
void registerLayout(const std::string & layout_type, Creator create_layout);
bool isComplex(const std::string & layout_type) const { return layout_complexity.at(layout_type); }
void registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex);
private:
using LayoutRegistry = std::unordered_map<std::string, Creator>;
LayoutRegistry registered_layouts;
using LayoutComplexity = std::unordered_map<std::string, bool>;
LayoutComplexity layout_complexity;
};
}

View File

@ -80,7 +80,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const DictionaryStructure & dict_struct,
Context & context) const
const Context & context) const
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);

View File

@ -31,7 +31,7 @@ public:
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context)>;
const Context & context)>;
DictionarySourceFactory();
@ -42,7 +42,7 @@ public:
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const DictionaryStructure & dict_struct,
Context & context) const;
const Context & context) const;
private:
using SourceRegistry = std::unordered_map<std::string, Creator>;

View File

@ -217,7 +217,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context) -> DictionarySourcePtr
const Context & context) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `executable` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};

View File

@ -56,7 +56,7 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context) -> DictionarySourcePtr
const Context & context) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `file` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};

View File

@ -724,7 +724,7 @@ void registerDictionaryFlat(DictionaryFactory & factory)
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("flat", create_layout);
factory.registerLayout("flat", create_layout, false);
}

View File

@ -188,7 +188,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context) -> DictionarySourcePtr
const Context & context) -> DictionarySourcePtr
{
if (dict_struct.has_expressions)
throw Exception{"Dictionary source of type `http` does not support attribute expressions", ErrorCodes::LOGICAL_ERROR};

View File

@ -787,8 +787,8 @@ void registerDictionaryHashed(DictionaryFactory & factory)
const bool sparse = name == "sparse_hashed";
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse);
};
factory.registerLayout("hashed", create_layout);
factory.registerLayout("sparse_hashed", create_layout);
factory.registerLayout("hashed", create_layout, false);
factory.registerLayout("sparse_hashed", create_layout, false);
}
}

View File

@ -691,7 +691,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("range_hashed", create_layout);
factory.registerLayout("range_hashed", create_layout, false);
}
}

View File

@ -767,7 +767,7 @@ void registerDictionaryTrie(DictionaryFactory & factory)
// This is specialised trie for storing IPv4 and IPv6 prefixes.
return std::make_unique<TrieDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
};
factory.registerLayout("ip_trie", create_layout);
factory.registerLayout("ip_trie", create_layout, true);
}
}

View File

@ -238,7 +238,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Block & sample_block,
Context & context) -> DictionarySourcePtr {
const Context & context) -> DictionarySourcePtr {
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));

View File

@ -0,0 +1,444 @@
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Poco/DOM/AutoPtr.h>
#include <Poco/DOM/Document.h>
#include <Poco/DOM/Element.h>
#include <Poco/DOM/Text.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <IO/WriteHelpers.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTFunction.h>
#include <Core/Names.h>
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
#include <Dictionaries/DictionaryFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DICTIONARY_DEFINITION;
}
/// There are a lot of code, but it's very simple and straightforward
/// We just convert
namespace
{
/// Get value from field and convert it to string.
/// Also remove quotes from strings.
String getUnescapedFieldString(const Field & field)
{
String string = applyVisitor(FieldVisitorToString(), field);
if (!string.empty() && string.front() == '\'' && string.back() == '\'')
return string.substr(1, string.size() - 2);
return string;
}
using namespace Poco;
using namespace Poco::XML;
/*
* Transforms next definition
* LIFETIME(MIN 10, MAX 100)
* to the next configuration
* <lifetime>
* <min>10</min>
* <max>100</max>
* </lifetime>
*/
void buildLifetimeConfiguration(
AutoPtr<Document> doc,
AutoPtr<Element> root,
const ASTDictionaryLifetime * lifetime)
{
AutoPtr<Element> lifetime_element(doc->createElement("lifetime"));
AutoPtr<Element> min_element(doc->createElement("min"));
AutoPtr<Element> max_element(doc->createElement("max"));
AutoPtr<Text> min_sec(doc->createTextNode(toString(lifetime->min_sec)));
min_element->appendChild(min_sec);
AutoPtr<Text> max_sec(doc->createTextNode(toString(lifetime->max_sec)));
max_element->appendChild(max_sec);
lifetime_element->appendChild(min_element);
lifetime_element->appendChild(max_element);
root->appendChild(lifetime_element);
}
/*
* Transforms next definition
* LAYOUT(FLAT())
* to the next configuration
* <layout>
* <flat/>
* </layout>
*
* And next definition
* LAYOUT(CACHE(SIZE_IN_CELLS 1000))
* to the next one
* <layout>
* <cache>
* <size_in_cells>1000</size_in_cells>
* </cache>
* </layout>
*/
void buildLayoutConfiguration(
AutoPtr<Document> doc,
AutoPtr<Element> root,
const ASTDictionaryLayout * layout)
{
AutoPtr<Element> layout_element(doc->createElement("layout"));
root->appendChild(layout_element);
AutoPtr<Element> layout_type_element(doc->createElement(layout->layout_type));
layout_element->appendChild(layout_type_element);
if (layout->parameter.has_value())
{
const auto & param = layout->parameter;
AutoPtr<Element> layout_type_parameter_element(doc->createElement(param->first));
const ASTLiteral & literal = param->second->as<const ASTLiteral &>();
AutoPtr<Text> value(doc->createTextNode(toString(literal.value.get<UInt64>())));
layout_type_parameter_element->appendChild(value);
layout_type_element->appendChild(layout_type_parameter_element);
}
}
/*
* Transforms next definition
* RANGE(MIN StartDate, MAX EndDate)
* to the next configuration
* <range_min><name>StartDate</name></range_min>
* <range_max><name>EndDate</name></range_max>
*/
void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTDictionaryRange * range)
{
// appends <key><name>value</name></key> to root
auto appendElem = [&doc, &root](const std::string & key, const std::string & value)
{
AutoPtr<Element> element(doc->createElement(key));
AutoPtr<Element> name(doc->createElement("name"));
AutoPtr<Text> text(doc->createTextNode(value));
name->appendChild(text);
element->appendChild(name);
root->appendChild(element);
};
appendElem("range_min", range->min_attr_name);
appendElem("range_max", range->max_attr_name);
}
/// Get primary key columns names from AST
Names getPrimaryKeyColumns(const ASTExpressionList * primary_key)
{
Names result;
const auto & children = primary_key->children;
for (size_t index = 0; index != children.size(); ++index)
{
const ASTIdentifier * key_part = children[index]->as<const ASTIdentifier>();
result.push_back(key_part->name);
}
return result;
}
/**
* Transofrms single dictionary attribute to configuration
* third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77
* to
* <attribute>
* <name>third_column</name>
* <type>UInt8</type>
* <null_value>2</null_value>
* <expression>(rand() % 100) * 77</expression>
* </attribute>
*/
void buildSingleAttribute(
AutoPtr<Document> doc,
AutoPtr<Element> root,
const ASTDictionaryAttributeDeclaration * dict_attr)
{
AutoPtr<Element> attribute_element(doc->createElement("attribute"));
root->appendChild(attribute_element);
AutoPtr<Element> name_element(doc->createElement("name"));
AutoPtr<Text> name(doc->createTextNode(dict_attr->name));
name_element->appendChild(name);
attribute_element->appendChild(name_element);
AutoPtr<Element> type_element(doc->createElement("type"));
AutoPtr<Text> type(doc->createTextNode(queryToString(dict_attr->type)));
type_element->appendChild(type);
attribute_element->appendChild(type_element);
AutoPtr<Element> null_value_element(doc->createElement("null_value"));
String null_value_str;
if (dict_attr->default_value)
null_value_str = queryToString(dict_attr->default_value);
AutoPtr<Text> null_value(doc->createTextNode(null_value_str));
null_value_element->appendChild(null_value);
attribute_element->appendChild(null_value_element);
if (dict_attr->expression != nullptr)
{
AutoPtr<Element> expression_element(doc->createElement("expression"));
AutoPtr<Text> expression(doc->createTextNode(queryToString(dict_attr->expression)));
expression_element->appendChild(expression);
attribute_element->appendChild(expression_element);
}
if (dict_attr->hierarchical)
{
AutoPtr<Element> hierarchical_element(doc->createElement("hierarchical"));
AutoPtr<Text> hierarchical(doc->createTextNode("true"));
hierarchical_element->appendChild(hierarchical);
attribute_element->appendChild(hierarchical_element);
}
if (dict_attr->injective)
{
AutoPtr<Element> injective_element(doc->createElement("injective"));
AutoPtr<Text> injective(doc->createTextNode("true"));
injective_element->appendChild(injective);
attribute_element->appendChild(injective_element);
}
if (dict_attr->is_object_id)
{
AutoPtr<Element> is_object_id_element(doc->createElement("is_object_id"));
AutoPtr<Text> is_object_id(doc->createTextNode("true"));
is_object_id_element->appendChild(is_object_id);
attribute_element->appendChild(is_object_id_element);
}
}
/**
* Transforms
* PRIMARY KEY Attr1 ,..., AttrN
* to the next configuration
* <id><name>Attr1</name></id>
* or
* <key>
* <attribute>
* <name>Attr1</name>
* <type>UInt8</type>
* </attribute>
* ...
* <attribute> fe
* </key>
*
*/
void buildPrimaryKeyConfiguration(
AutoPtr<Document> doc,
AutoPtr<Element> root,
bool complex,
const Names & key_names,
const ASTExpressionList * dictionary_attributes)
{
if (!complex)
{
if (key_names.size() != 1)
throw Exception("Primary key for simple dictionary must contain exactly one element",
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
AutoPtr<Element> id_element(doc->createElement("id"));
root->appendChild(id_element);
AutoPtr<Element> name_element(doc->createElement("name"));
id_element->appendChild(name_element);
AutoPtr<Text> name(doc->createTextNode(*key_names.begin()));
name_element->appendChild(name);
}
else
{
const auto & children = dictionary_attributes->children;
if (children.size() < key_names.size())
throw Exception(
"Primary key fields count is more, than dictionary attributes count.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
AutoPtr<Element> key_element(doc->createElement("key"));
root->appendChild(key_element);
for (const auto & key_name : key_names)
{
bool found = false;
for (const auto & attr : children)
{
const ASTDictionaryAttributeDeclaration * dict_attr = attr->as<const ASTDictionaryAttributeDeclaration>();
if (dict_attr->name == key_name)
{
found = true;
buildSingleAttribute(doc, key_element, dict_attr);
break;
}
}
if (!found)
throw Exception(
"Primary key field '" + key_name + "' not found among attributes.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
}
}
}
/**
* Transforms list of ASTDictionaryAttributeDeclarations to list of dictionary attributes
*/
std::unordered_set<std::string> buildDictionaryAttributesConfiguration(
AutoPtr<Document> doc,
AutoPtr<Element> root,
const ASTExpressionList * dictionary_attributes,
const Names & key_columns)
{
const auto & children = dictionary_attributes->children;
std::unordered_set<std::string> dictionary_attributes_names;
for (size_t i = 0; i < children.size(); ++i)
{
const ASTDictionaryAttributeDeclaration * dict_attr = children[i]->as<const ASTDictionaryAttributeDeclaration>();
if (!dict_attr->type)
throw Exception("Dictionary attribute must has type", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
dictionary_attributes_names.insert(dict_attr->name);
if (std::find(key_columns.begin(), key_columns.end(), dict_attr->name) == key_columns.end())
buildSingleAttribute(doc, root, dict_attr);
}
return dictionary_attributes_names;
}
/** Transform function with key-value arguments to configuration
* (used for source transformation)
*/
void buildConfigurationFromFunctionWithKeyValueArguments(
AutoPtr<Document> doc,
AutoPtr<Element> root,
const ASTExpressionList * ast_expr_list)
{
const auto & children = ast_expr_list->children;
for (size_t i = 0; i != children.size(); ++i)
{
const ASTPair * pair = children[i]->as<const ASTPair>();
AutoPtr<Element> current_xml_element(doc->createElement(pair->first));
root->appendChild(current_xml_element);
if (auto identifier = pair->second->as<const ASTIdentifier>(); identifier)
{
AutoPtr<Text> value(doc->createTextNode(identifier->name));
current_xml_element->appendChild(value);
}
else if (auto literal = pair->second->as<const ASTLiteral>(); literal)
{
AutoPtr<Text> value(doc->createTextNode(getUnescapedFieldString(literal->value)));
current_xml_element->appendChild(value);
}
else if (auto list = pair->second->as<const ASTExpressionList>(); list)
{
buildConfigurationFromFunctionWithKeyValueArguments(doc, current_xml_element, list);
}
else
{
throw Exception(
"Incorrect ASTPair contains wrong value, should be literal, identifier or list",
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
}
}
}
/** Build source definition from ast.
* SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))
* to
* <source>
* <mysql>
* <host>localhost</host>
* ...
* <replica>
* <host>127.0.0.1</host>
* ...
* </replica>
* </mysql>
* </source>
*/
void buildSourceConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTFunctionWithKeyValueArguments * source)
{
AutoPtr<Element> outer_element(doc->createElement("source"));
root->appendChild(outer_element);
AutoPtr<Element> source_element(doc->createElement(source->name));
outer_element->appendChild(source_element);
buildConfigurationFromFunctionWithKeyValueArguments(doc, source_element, source->elements->as<const ASTExpressionList>());
}
/** Check all AST fields are filled, throws exception
* in other case
*/
void checkAST(const ASTCreateQuery & query)
{
if (!query.is_dictionary || query.dictionary == nullptr)
throw Exception("Cannot convert dictionary to configuration from non-dictionary AST.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
if (query.dictionary_attributes_list == nullptr || query.dictionary_attributes_list->children.empty())
throw Exception("Cannot create dictionary with empty attributes list", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
if (query.dictionary->layout == nullptr)
throw Exception("Cannot create dictionary with empty layout", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
if (query.dictionary->lifetime == nullptr)
throw Exception("Cannot create dictionary with empty lifetime", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
if (query.dictionary->primary_key == nullptr)
throw Exception("Cannot create dictionary without primary key", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
if (query.dictionary->source == nullptr)
throw Exception("Cannot create dictionary with empty source", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
/// Range can be empty
}
void checkPrimaryKey(const std::unordered_set<std::string> & all_attrs, const Names & key_attrs)
{
for (const auto & key_attr : key_attrs)
if (all_attrs.count(key_attr) == 0)
throw Exception("Unknown key attribute '" + key_attr + "'", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
}
}
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query)
{
checkAST(query);
AutoPtr<Poco::XML::Document> xml_document(new Poco::XML::Document());
AutoPtr<Poco::XML::Element> document_root(xml_document->createElement("dictionaries"));
xml_document->appendChild(document_root);
AutoPtr<Poco::XML::Element> current_dictionary(xml_document->createElement("dictionary"));
document_root->appendChild(current_dictionary);
AutoPtr<Poco::Util::XMLConfiguration> conf(new Poco::Util::XMLConfiguration());
AutoPtr<Poco::XML::Element> name_element(xml_document->createElement("name"));
current_dictionary->appendChild(name_element);
AutoPtr<Text> name(xml_document->createTextNode(query.database + "." + query.table));
name_element->appendChild(name);
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
current_dictionary->appendChild(structure_element);
Names pk_attrs = getPrimaryKeyColumns(query.dictionary->primary_key);
auto dictionary_layout = query.dictionary->layout;
bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type);
auto all_attr_names = buildDictionaryAttributesConfiguration(xml_document, structure_element, query.dictionary_attributes_list, pk_attrs);
checkPrimaryKey(all_attr_names, pk_attrs);
buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list);
buildLayoutConfiguration(xml_document, current_dictionary, dictionary_layout);
buildSourceConfiguration(xml_document, current_dictionary, query.dictionary->source);
buildLifetimeConfiguration(xml_document, current_dictionary, query.dictionary->lifetime);
if (query.dictionary->range)
buildRangeConfiguration(xml_document, structure_element, query.dictionary->range);
conf->load(xml_document);
return conf;
}
}

View File

@ -0,0 +1,15 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <Parsers/ASTCreateQuery.h>
namespace DB
{
using DictionaryConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/// Convert dictionary AST to Poco::AbstractConfiguration
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
/// Can throw exception if query is ill-formed
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query);
}

View File

@ -0,0 +1,224 @@
#include <iostream>
#include <sstream>
#include <Core/Types.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/DumpASTNode.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ParserDictionary.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserTablePropertiesQuery.h>
#include <Parsers/TablePropertiesQueriesASTs.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
#include <Dictionaries/registerDictionaries.h>
#include <gtest/gtest.h>
using namespace DB;
static bool registered = false;
/// For debug
std::string configurationToString(const DictionaryConfigurationPtr & config)
{
const Poco::Util::XMLConfiguration * xml_config = dynamic_cast<const Poco::Util::XMLConfiguration *>(config.get());
std::ostringstream oss;
xml_config->save(oss);
return oss.str();
}
TEST(ConvertDictionaryAST, SimpleDictConfiguration)
{
if (!registered)
{
registerDictionaries();
registered = true;
}
String input = " CREATE DICTIONARY test.dict1"
" ("
" key_column UInt64 DEFAULT 0,"
" second_column UInt8 DEFAULT 1,"
" third_column UInt8 DEFAULT 2"
" )"
" PRIMARY KEY key_column"
" SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' PASSWORD '' DB 'test' TABLE 'table_for_dict'))"
" LAYOUT(FLAT())"
" LIFETIME(MIN 1 MAX 10)"
" RANGE(MIN second_column MAX third_column)";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
/// name
EXPECT_EQ(config->getString("dictionary.name"), "test.dict1");
/// lifetime
EXPECT_EQ(config->getInt("dictionary.lifetime.min"), 1);
EXPECT_EQ(config->getInt("dictionary.lifetime.max"), 10);
/// range
EXPECT_EQ(config->getString("dictionary.structure.range_min"), "second_column");
EXPECT_EQ(config->getString("dictionary.structure.range_max"), "third_column");
/// source
EXPECT_EQ(config->getString("dictionary.source.clickhouse.host"), "localhost");
EXPECT_EQ(config->getInt("dictionary.source.clickhouse.port"), 9000);
EXPECT_EQ(config->getString("dictionary.source.clickhouse.user"), "default");
EXPECT_EQ(config->getString("dictionary.source.clickhouse.password"), "");
EXPECT_EQ(config->getString("dictionary.source.clickhouse.db"), "test");
EXPECT_EQ(config->getString("dictionary.source.clickhouse.table"), "table_for_dict");
/// attributes and key
Poco::Util::AbstractConfiguration::Keys keys;
config->keys("dictionary.structure", keys);
EXPECT_EQ(keys.size(), 5); /// + ranged keys
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".name"), "second_column");
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".type"), "UInt8");
EXPECT_EQ(config->getInt("dictionary.structure." + keys[0] + ".null_value"), 1);
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".name"), "third_column");
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".type"), "UInt8");
EXPECT_EQ(config->getInt("dictionary.structure." + keys[1] + ".null_value"), 2);
EXPECT_EQ(keys[2], "id");
EXPECT_EQ(config->getString("dictionary.structure." + keys[2] + ".name"), "key_column");
/// layout
EXPECT_TRUE(config->has("dictionary.layout.flat"));
}
TEST(ConvertDictionaryAST, TrickyAttributes)
{
if (!registered)
{
registerDictionaries();
registered = true;
}
String input = " CREATE DICTIONARY dict2"
" ("
" key_column UInt64 IS_OBJECT_ID,"
" second_column UInt8 HIERARCHICAL INJECTIVE,"
" third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77"
" )"
" PRIMARY KEY key_column"
" LAYOUT(hashed())"
" LIFETIME(MIN 1 MAX 10)"
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
Poco::Util::AbstractConfiguration::Keys keys;
config->keys("dictionary.structure", keys);
EXPECT_EQ(keys.size(), 3);
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".name"), "second_column");
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".type"), "UInt8");
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".null_value"), "");
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".hierarchical"), "true");
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".injective"), "true");
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".name"), "third_column");
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".type"), "UInt8");
EXPECT_EQ(config->getInt("dictionary.structure." + keys[1] + ".null_value"), 2);
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".expression"), "(rand() % 100) * 77");
EXPECT_EQ(keys[2], "id");
EXPECT_EQ(config->getString("dictionary.structure." + keys[2] + ".name"), "key_column");
}
TEST(ConvertDictionaryAST, ComplexKeyAndLayoutWithParams)
{
if (!registered)
{
registerDictionaries();
registered = true;
}
String input = " CREATE DICTIONARY dict4"
" ("
" key_column1 String,"
" key_column2 UInt64,"
" third_column UInt8,"
" fourth_column UInt8"
" )"
" PRIMARY KEY key_column1, key_column2"
" SOURCE(MYSQL())"
" LAYOUT(COMPLEX_KEY_CACHE(size_in_cells 50))"
" LIFETIME(MIN 1 MAX 10)";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
Poco::Util::AbstractConfiguration::Keys keys;
config->keys("dictionary.structure.key", keys);
EXPECT_EQ(keys.size(), 2);
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[0] + ".name"), "key_column1");
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[0] + ".type"), "String");
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[1] + ".name"), "key_column2");
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[1] + ".type"), "UInt64");
Poco::Util::AbstractConfiguration::Keys attrs;
config->keys("dictionary.structure", attrs);
EXPECT_EQ(attrs.size(), 3);
EXPECT_EQ(config->getString("dictionary.structure." + attrs[0] + ".name"), "third_column");
EXPECT_EQ(config->getString("dictionary.structure." + attrs[0] + ".type"), "UInt8");
EXPECT_EQ(config->getString("dictionary.structure." + attrs[1] + ".name"), "fourth_column");
EXPECT_EQ(config->getString("dictionary.structure." + attrs[1] + ".type"), "UInt8");
EXPECT_EQ(attrs[2], "key");
EXPECT_EQ(config->getInt("dictionary.layout.complex_key_cache.size_in_cells"), 50);
}
TEST(ConvertDictionaryAST, ComplexSource)
{
if (!registered)
{
registerDictionaries();
registered = true;
}
String input = " CREATE DICTIONARY dict4"
" ("
" key_column UInt64,"
" second_column UInt8,"
" third_column UInt8"
" )"
" PRIMARY KEY key_column"
" SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))"
" LAYOUT(CACHE(size_in_cells 50))"
" LIFETIME(MIN 1 MAX 10)"
" RANGE(MIN second_column MAX third_column)";
ParserCreateDictionaryQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
/// source
EXPECT_EQ(config->getString("dictionary.source.mysql.host"), "localhost");
EXPECT_EQ(config->getInt("dictionary.source.mysql.port"), 9000);
EXPECT_EQ(config->getString("dictionary.source.mysql.user"), "default");
EXPECT_EQ(config->getString("dictionary.source.mysql.password"), "");
EXPECT_EQ(config->getString("dictionary.source.mysql.replica.host"), "127.0.0.1");
EXPECT_EQ(config->getInt("dictionary.source.mysql.replica.priority"), 1);
}

146
dbms/src/Functions/CRC.cpp Normal file
View File

@ -0,0 +1,146 @@
#include <zlib.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringOrArrayToT.h>
namespace
{
template <class T>
struct CRCBase
{
T tab[256];
CRCBase(T polynomial)
{
for (size_t i = 0; i < 256; ++i)
{
T c = i;
for (size_t j = 0; j < 8; ++j)
{
c = c & 1 ? polynomial ^ (c >> 1) : c >> 1;
}
tab[i] = c;
}
}
};
template <class T, T polynomial>
struct CRCImpl
{
using ReturnType = T;
static T make_crc(const unsigned char *buf, size_t size)
{
static CRCBase<ReturnType> base(polynomial);
T i, crc;
crc = 0;
for (i = 0; i < size; i++)
{
crc = base.tab[(crc ^ buf[i]) & 0xff] ^ (crc >> 8);
}
return crc;
}
};
static constexpr UInt64 CRC64_ECMA = 0xc96c5795d7870f42ULL;
struct CRC64ECMAImpl : public CRCImpl<UInt64, CRC64_ECMA>
{
static constexpr auto name = "CRC64";
};
static constexpr UInt32 CRC32_IEEE = 0xedb88320;
struct CRC32IEEEImpl : public CRCImpl<UInt32, CRC32_IEEE>
{
static constexpr auto name = "CRC32IEEE";
};
struct CRC32ZLIBImpl
{
using ReturnType = UInt32;
static constexpr auto name = "CRC32";
static UInt32 make_crc(const unsigned char *buf, size_t size)
{ return crc32_z(0L, buf, size); }
};
} // \anonymous
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
template <class Impl>
struct CRCFunctionWrapper
{
static constexpr auto is_fixed_to_constant = true;
using ReturnType = typename Impl::ReturnType;
static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<ReturnType> & res)
{
size_t size = offsets.size();
ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
res[i] = do_crc(data, prev_offset, offsets[i] - prev_offset - 1);
prev_offset = offsets[i];
}
}
static void vector_fixed_to_constant(const ColumnString::Chars & data, size_t n, ReturnType & res) { res = do_crc(data, 0, n); }
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<ReturnType> & res)
{
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
{
res[i] = do_crc(data, i * n, n);
}
}
[[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray<ReturnType> & /*res*/)
{
throw Exception("Cannot apply function " + std::string(Impl::name) + " to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
private:
static ReturnType do_crc(const ColumnString::Chars & buf, size_t offset, size_t size)
{
const unsigned char * p = reinterpret_cast<const unsigned char *>(&buf[0]) + offset;
return Impl::make_crc(p, size);
}
};
template <class T>
using FunctionCRC = FunctionStringOrArrayToT<CRCFunctionWrapper<T>, T, typename T::ReturnType>;
// The same as IEEE variant, but uses 0xffffffff as initial value
// This is the default
//
// (And zlib is used here, since it has optimized version)
using FunctionCRC32ZLIB = FunctionCRC<CRC32ZLIBImpl>;
// Uses CRC-32-IEEE 802.3 polynomial
using FunctionCRC32IEEE = FunctionCRC<CRC32IEEEImpl>;
// Uses CRC-64-ECMA polynomial
using FunctionCRC64ECMA = FunctionCRC<CRC64ECMAImpl>;
template <class T>
void registerFunctionCRCImpl(FunctionFactory & factory)
{
factory.registerFunction<T>(T::name, FunctionFactory::CaseInsensitive);
}
void registerFunctionCRC(FunctionFactory & factory)
{
registerFunctionCRCImpl<FunctionCRC32ZLIB>(factory);
registerFunctionCRCImpl<FunctionCRC32IEEE>(factory);
registerFunctionCRCImpl<FunctionCRC64ECMA>(factory);
}
}

View File

@ -1,68 +0,0 @@
#include <zlib.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionStringOrArrayToT.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/** Calculates the CRC32 of a string
*/
struct CRC32Impl
{
static constexpr auto is_fixed_to_constant = true;
static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt32> & res)
{
size_t size = offsets.size();
ColumnString::Offset prev_offset = 0;
for (size_t i = 0; i < size; ++i)
{
res[i] = do_crc32(data, prev_offset, offsets[i] - prev_offset - 1);
prev_offset = offsets[i];
}
}
static void vector_fixed_to_constant(const ColumnString::Chars & data, size_t n, UInt32 & res) { res = do_crc32(data, 0, n); }
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt32> & res)
{
size_t size = data.size() / n;
for (size_t i = 0; i < size; ++i)
{
res[i] = do_crc32(data, i * n, n);
}
}
[[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray<UInt32> & /*res*/)
{
throw Exception("Cannot apply function CRC32 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
private:
static uint32_t do_crc32(const ColumnString::Chars & buf, size_t offset, size_t size)
{
const unsigned char * p = reinterpret_cast<const unsigned char *>(&buf[0]) + offset;
return crc32(0L, p, size);
}
};
struct NameCRC32
{
static constexpr auto name = "CRC32";
};
using FunctionCRC32 = FunctionStringOrArrayToT<CRC32Impl, NameCRC32, UInt32>;
void registerFunctionCRC32(FunctionFactory & factory)
{
factory.registerFunction<FunctionCRC32>(NameCRC32::name, FunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,68 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Interpreters/Context.h>
#include <Common/Macros.h>
#include <Core/Field.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_COLUMN;
}
/** Get scalar value of sub queries from query context via IAST::Hash.
*/
class FunctionGetScalar : public IFunction
{
public:
static constexpr auto name = "__getScalar";
static FunctionPtr create(const Context & context)
{
return std::make_shared<FunctionGetScalar>(context);
}
FunctionGetScalar(const Context & context_) : context(context_) {}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 1;
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1 || !isString(arguments[0].type) || !isColumnConst(*arguments[0].column))
throw Exception("Function " + getName() + " accepts one const string argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto scalar_name = assert_cast<const ColumnConst &>(*arguments[0].column).getField().get<String>();
scalar = context.getScalar(scalar_name).getByPosition(0);
return scalar.type;
}
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = ColumnConst::create(scalar.column, input_rows_count);
}
private:
mutable ColumnWithTypeAndName scalar;
const Context & context;
};
void registerFunctionGetScalar(FunctionFactory & factory)
{
factory.registerFunction<FunctionGetScalar>();
}
}

View File

@ -53,6 +53,7 @@ void registerFunctionBasename(FunctionFactory &);
void registerFunctionTransform(FunctionFactory &);
void registerFunctionGetMacro(FunctionFactory &);
void registerFunctionRandomASKII(FunctionFactory &);
void registerFunctionGetScalar(FunctionFactory &);
#if USE_ICU
void registerFunctionConvertCharset(FunctionFactory &);
@ -108,6 +109,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
registerFunctionTransform(factory);
registerFunctionGetMacro(factory);
registerFunctionRandomASKII(factory);
registerFunctionGetScalar(factory);
#if USE_ICU
registerFunctionConvertCharset(factory);

View File

@ -20,7 +20,7 @@ void registerFunctionReverseUTF8(FunctionFactory &);
void registerFunctionsConcat(FunctionFactory &);
void registerFunctionFormat(FunctionFactory &);
void registerFunctionSubstring(FunctionFactory &);
void registerFunctionCRC32(FunctionFactory &);
void registerFunctionCRC(FunctionFactory &);
void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &);
void registerFunctionStartsWith(FunctionFactory &);
void registerFunctionEndsWith(FunctionFactory &);
@ -47,7 +47,7 @@ void registerFunctionsString(FunctionFactory & factory)
registerFunctionLowerUTF8(factory);
registerFunctionUpperUTF8(factory);
registerFunctionReverse(factory);
registerFunctionCRC32(factory);
registerFunctionCRC(factory);
registerFunctionReverseUTF8(factory);
registerFunctionsConcat(factory);
registerFunctionFormat(factory);

View File

@ -23,7 +23,7 @@ template <typename F>
inline void forEachTable(Context & context, F && f)
{
for (auto & elem : context.getDatabases())
for (auto iterator = elem.second->getIterator(context); iterator->isValid(); iterator->next())
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
f(iterator->table());
}

View File

@ -29,7 +29,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Interpreters/ActionsVisitor.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/Set.h>

View File

@ -167,7 +167,7 @@ void AsynchronousMetrics::update()
/// Lazy database can not contain MergeTree tables
if (db.second->getEngineName() == "Lazy")
continue;
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
{
++total_number_of_tables;
auto & table = iterator->table();

View File

@ -33,11 +33,13 @@ SelectStreamFactory::SelectStreamFactory(
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
QualifiedTableName main_table_,
const Scalars & scalars_,
const Tables & external_tables_)
: header(header_),
processed_stage{processed_stage_},
main_table(std::move(main_table_)),
table_func_ptr{nullptr},
scalars{scalars_},
external_tables{external_tables_}
{
}
@ -46,10 +48,12 @@ SelectStreamFactory::SelectStreamFactory(
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
ASTPtr table_func_ptr_,
const Scalars & scalars_,
const Tables & external_tables_)
: header(header_),
processed_stage{processed_stage_},
table_func_ptr{table_func_ptr_},
scalars{scalars_},
external_tables{external_tables_}
{
}
@ -92,7 +96,8 @@ void SelectStreamFactory::createForShard(
auto emplace_remote_stream = [&]()
{
auto stream = std::make_shared<RemoteBlockInputStream>(shard_info.pool, query, header, context, nullptr, throttler, external_tables, processed_stage);
auto stream = std::make_shared<RemoteBlockInputStream>(
shard_info.pool, query, header, context, nullptr, throttler, scalars, external_tables, processed_stage);
stream->setPoolMode(PoolMode::GET_MANY);
if (!table_func_ptr)
stream->setMainTable(main_table);
@ -190,8 +195,8 @@ void SelectStreamFactory::createForShard(
auto lazily_create_stream = [
pool = shard_info.pool, shard_num = shard_info.shard_num, query, header = header, query_ast, context, throttler,
main_table = main_table, table_func_ptr = table_func_ptr, external_tables = external_tables, stage = processed_stage,
local_delay]()
main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables,
stage = processed_stage, local_delay]()
-> BlockInputStreamPtr
{
auto current_settings = context.getSettingsRef();
@ -233,7 +238,7 @@ void SelectStreamFactory::createForShard(
connections.emplace_back(std::move(try_result.entry));
return std::make_shared<RemoteBlockInputStream>(
std::move(connections), query, header, context, nullptr, throttler, external_tables, stage);
std::move(connections), query, header, context, nullptr, throttler, scalars, external_tables, stage);
}
};

View File

@ -18,6 +18,7 @@ public:
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
QualifiedTableName main_table_,
const Scalars & scalars_,
const Tables & external_tables);
/// TableFunction in a query.
@ -25,6 +26,7 @@ public:
const Block & header_,
QueryProcessingStage::Enum processed_stage_,
ASTPtr table_func_ptr_,
const Scalars & scalars_,
const Tables & external_tables_);
void createForShard(
@ -38,6 +40,7 @@ private:
QueryProcessingStage::Enum processed_stage;
QualifiedTableName main_table;
ASTPtr table_func_ptr;
Scalars scalars;
Tables external_tables;
};

View File

@ -164,7 +164,8 @@ size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vector<const ASTIde
/// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor.
/// In this case it saves membership in IdentifierSemantic.
size_t membership = IdentifierSemantic::getMembership(*identifier);
auto opt = IdentifierSemantic::getMembership(*identifier);
size_t membership = opt ? (*opt + 1) : 0;
if (!membership)
{

View File

@ -30,6 +30,7 @@
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
#include <Interpreters/EmbeddedDictionaries.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ExpressionActions.h>
@ -88,6 +89,8 @@ namespace ErrorCodes
extern const int SESSION_IS_LOCKED;
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int LOGICAL_ERROR;
extern const int SCALAR_ALREADY_EXISTS;
extern const int UNKNOWN_SCALAR;
}
@ -191,7 +194,7 @@ struct ContextShared
bool shutdown_called = false;
/// Do not allow simultaneous execution of DDL requests on the same table.
/// database -> table -> (mutex, counter), counter: how many threads are running a query on the table at the same time
/// database -> object -> (mutex, counter), counter: how many threads are running a query on the table at the same time
/// For the duration of the operation, an element is placed here, and an object is returned,
/// which deletes the element in the destructor when counter becomes zero.
/// In case the element already exists, waits, when query will be executed in other thread. See class DDLGuard below.
@ -792,6 +795,16 @@ bool Context::isTableExist(const String & database_name, const String & table_na
&& it->second->isTableExist(*this, table_name);
}
bool Context::isDictionaryExists(const String & database_name, const String & dictionary_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRightsImpl(db);
Databases::const_iterator it = shared->databases.find(db);
return shared->databases.end() != it && it->second->isDictionaryExist(*this, dictionary_name);
}
bool Context::isDatabaseExist(const String & database_name) const
{
@ -807,22 +820,6 @@ bool Context::isExternalTableExist(const String & table_name) const
}
void Context::assertTableExists(const String & database_name, const String & table_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
checkDatabaseAccessRightsImpl(db);
Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() == it)
throw Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
if (!it->second->isTableExist(*this, table_name))
throw Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
}
void Context::assertTableDoesntExist(const String & database_name, const String & table_name, bool check_database_access_rights) const
{
auto lock = getLock();
@ -862,6 +859,21 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const
}
const Scalars & Context::getScalars() const
{
return scalars;
}
const Block & Context::getScalar(const String & name) const
{
auto it = scalars.find(name);
if (scalars.end() == it)
throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::UNKNOWN_SCALAR);
return it->second;
}
Tables Context::getExternalTables() const
{
auto lock = getLock();
@ -959,6 +971,19 @@ void Context::addExternalTable(const String & table_name, const StoragePtr & sto
external_tables[table_name] = std::pair(storage, ast);
}
void Context::addScalar(const String & name, const Block & block)
{
scalars[name] = block;
}
bool Context::hasScalar(const String & name) const
{
return scalars.count(name);
}
StoragePtr Context::tryRemoveExternalTable(const String & table_name)
{
TableAndCreateASTs::const_iterator it = external_tables.find(table_name);
@ -1046,9 +1071,10 @@ void Context::addDatabase(const String & database_name, const DatabasePtr & data
DatabasePtr Context::detachDatabase(const String & database_name)
{
auto lock = getLock();
auto res = getDatabase(database_name);
getExternalDictionariesLoader().removeConfigRepository(database_name);
shared->databases.erase(database_name);
return res;
}
@ -1063,6 +1089,17 @@ ASTPtr Context::getCreateTableQuery(const String & database_name, const String &
return shared->databases[db]->getCreateTableQuery(*this, table_name);
}
ASTPtr Context::getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const
{
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
assertDatabaseExists(db);
return shared->databases[db]->getCreateDictionaryQuery(*this, dictionary_name);
}
ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const
{
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
@ -1308,21 +1345,13 @@ EmbeddedDictionaries & Context::getEmbeddedDictionaries()
const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const
{
{
std::lock_guard lock(shared->external_dictionaries_mutex);
if (shared->external_dictionaries_loader)
return *shared->external_dictionaries_loader;
}
const auto & config = getConfigRef();
std::lock_guard lock(shared->external_dictionaries_mutex);
if (!shared->external_dictionaries_loader)
{
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config, "dictionaries_config");
shared->external_dictionaries_loader.emplace(std::move(config_repository), *this->global_context);
shared->external_dictionaries_loader.emplace(*this->global_context);
}
return *shared->external_dictionaries_loader;
}
@ -1341,8 +1370,7 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const
if (!this->global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(getConfigRef(), "models_config");
shared->external_models_loader.emplace(std::move(config_repository), *this->global_context);
shared->external_models_loader.emplace(*this->global_context);
}
return *shared->external_models_loader;
}
@ -2039,7 +2067,7 @@ void Context::dropCompiledExpressionCache() const
#endif
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd)
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
{
auto lock = getLock();
shared->bridge_commands.emplace_back(std::move(cmd));

View File

@ -105,6 +105,9 @@ using InputInitializer = std::function<void(Context &, const StoragePtr &)>;
/// Callback for reading blocks of data from client for function input()
using InputBlocksReader = std::function<Block(Context &)>;
/// Scalar results of sub queries
using Scalars = std::map<String, Block>;
/// An empty interface for an arbitrary object that may be attached by a shared pointer
/// to query context, when using ClickHouse as a library.
struct IHostContext
@ -144,6 +147,7 @@ private:
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
TableAndCreateASTs external_tables; /// Temporary tables.
Scalars scalars;
StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
Context * query_context = nullptr;
@ -248,9 +252,9 @@ public:
/// Checking the existence of the table/database. Database can be empty - in this case the current database is used.
bool isTableExist(const String & database_name, const String & table_name) const;
bool isDatabaseExist(const String & database_name) const;
bool isDictionaryExists(const String & database_name, const String & dictionary_name) const;
bool isExternalTableExist(const String & table_name) const;
bool hasDatabaseAccessRights(const String & database_name) const;
void assertTableExists(const String & database_name, const String & table_name) const;
bool hasDictionaryAccessRights(const String & dictionary_name) const;
@ -264,11 +268,15 @@ public:
void assertDatabaseDoesntExist(const String & database_name) const;
void checkDatabaseAccessRights(const std::string & database_name) const;
const Scalars & getScalars() const;
const Block & getScalar(const String & name) const;
Tables getExternalTables() const;
StoragePtr tryGetExternalTable(const String & table_name) const;
StoragePtr getTable(const String & database_name, const String & table_name) const;
StoragePtr tryGetTable(const String & database_name, const String & table_name) const;
void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {});
void addScalar(const String & name, const Block & block);
bool hasScalar(const String & name) const;
StoragePtr tryRemoveExternalTable(const String & table_name);
StoragePtr executeTableFunction(const ASTPtr & table_expression);
@ -355,6 +363,7 @@ public:
ASTPtr getCreateTableQuery(const String & database_name, const String & table_name) const;
ASTPtr getCreateExternalTableQuery(const String & table_name) const;
ASTPtr getCreateDatabaseQuery(const String & database_name) const;
ASTPtr getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const;
const DatabasePtr getDatabase(const String & database_name) const;
DatabasePtr getDatabase(const String & database_name);
@ -544,7 +553,7 @@ public:
#endif
/// Add started bridge command. It will be killed after context destruction
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd);
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const;
IHostContextPtr & getHostContext();
const IHostContextPtr & getHostContext() const;

View File

@ -4,7 +4,7 @@
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/QueryNormalizer.h> // for functionIsInOperator
#include <Interpreters/misc.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
@ -90,9 +90,12 @@ public:
using TypeToVisit = const ASTFunction;
CheckExpressionVisitorData(const std::vector<JoinedTable> & tables_)
: tables(tables_)
: joined_tables(tables_)
, ands_only(true)
{}
{
for (auto & joined : joined_tables)
tables.push_back(joined.table);
}
void visit(const ASTFunction & node, const ASTPtr & ast)
{
@ -156,7 +159,8 @@ public:
}
private:
const std::vector<JoinedTable> & tables;
const std::vector<JoinedTable> & joined_tables;
std::vector<DatabaseAndTableWithAlias> tables;
std::map<size_t, std::vector<ASTPtr>> asts_to_join_on;
bool ands_only;
@ -180,31 +184,16 @@ private:
/// @return table position to attach expression to or 0.
size_t checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right)
{
/// {best_match, best_table_pos}
std::pair<size_t, size_t> left_best{0, 0};
std::pair<size_t, size_t> right_best{0, 0};
size_t left_table_pos = 0;
bool left_match = IdentifierSemantic::chooseTable(left, tables, left_table_pos);
for (size_t i = 0; i < tables.size(); ++i)
size_t right_table_pos = 0;
bool right_match = IdentifierSemantic::chooseTable(right, tables, right_table_pos);
if (left_match && right_match && (left_table_pos != right_table_pos))
{
size_t match = IdentifierSemantic::canReferColumnToTable(left, tables[i].table);
if (match > left_best.first)
{
left_best.first = match;
left_best.second = i;
}
match = IdentifierSemantic::canReferColumnToTable(right, tables[i].table);
if (match > right_best.first)
{
right_best.first = match;
right_best.second = i;
}
}
if (left_best.first && right_best.first && (left_best.second != right_best.second))
{
size_t table_pos = std::max(left_best.second, right_best.second);
if (tables[table_pos].canAttachOnExpression())
size_t table_pos = std::max(left_table_pos, right_table_pos);
if (joined_tables[table_pos].canAttachOnExpression())
return table_pos;
}
return 0;
@ -212,20 +201,10 @@ private:
size_t checkIdentifier(const ASTIdentifier & identifier)
{
size_t best_match = 0;
size_t best_table_pos = 0;
bool match = IdentifierSemantic::chooseTable(identifier, tables, best_table_pos);
for (size_t i = 0; i < tables.size(); ++i)
{
size_t match = IdentifierSemantic::canReferColumnToTable(identifier, tables[i].table);
if (match > best_match)
{
best_match = match;
best_table_pos = i;
}
}
if (best_match && tables[best_table_pos].canAttachOnExpression())
if (match && joined_tables[best_table_pos].canAttachOnExpression())
return best_table_pos;
return 0;
}

View File

@ -6,14 +6,17 @@
#include <Parsers/ASTExpressionList.h>
#include <Interpreters/Context.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
#include <Interpreters/addTypeConversionToAST.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/materializeBlock.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnTuple.h>
namespace DB
{
@ -53,69 +56,98 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data)
visit(*t, ast, data);
}
/// Converting to literal values might take a fair amount of overhead when the value is large, (e.g.
/// Array, BitMap, etc.), This conversion is required for constant folding, index lookup, branch
/// elimination. However, these optimizations should never be related to large values, thus we
/// blacklist them here.
static bool worthConvertingToLiteral(const Block & scalar)
{
auto scalar_type_name = scalar.safeGetByPosition(0).type->getFamilyName();
std::set<String> useless_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"};
return !useless_literal_types.count(scalar_type_name);
}
void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr & ast, Data & data)
{
Context subquery_context = data.context;
Settings subquery_settings = data.context.getSettings();
subquery_settings.max_result_rows = 1;
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
auto hash = subquery.getTreeHash();
auto scalar_query_hash_str = toString(hash.first) + "_" + toString(hash.second);
ASTPtr subquery_select = subquery.children.at(0);
BlockIO res = InterpreterSelectWithUnionQuery(
subquery_select, subquery_context, SelectQueryOptions(QueryProcessingStage::Complete, data.subquery_depth + 1)).execute();
Block block;
try
Block scalar;
if (data.context.hasQueryContext() && data.context.getQueryContext().hasScalar(scalar_query_hash_str))
scalar = data.context.getQueryContext().getScalar(scalar_query_hash_str);
else if (data.scalars.count(scalar_query_hash_str))
scalar = data.scalars[scalar_query_hash_str];
else
{
block = res.in->read();
Context subquery_context = data.context;
Settings subquery_settings = data.context.getSettings();
subquery_settings.max_result_rows = 1;
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
if (!block)
ASTPtr subquery_select = subquery.children.at(0);
BlockIO res = InterpreterSelectWithUnionQuery(
subquery_select, subquery_context, SelectQueryOptions(QueryProcessingStage::Complete, data.subquery_depth + 1)).execute();
Block block;
try
{
/// Interpret subquery with empty result as Null literal
auto ast_new = std::make_unique<ASTLiteral>(Null());
ast_new->setAlias(ast->tryGetAlias());
ast = std::move(ast_new);
return;
block = res.in->read();
if (!block)
{
/// Interpret subquery with empty result as Null literal
auto ast_new = std::make_unique<ASTLiteral>(Null());
ast_new->setAlias(ast->tryGetAlias());
ast = std::move(ast_new);
return;
}
if (block.rows() != 1 || res.in->read())
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
}
catch (const Exception & e)
{
if (e.code() == ErrorCodes::TOO_MANY_ROWS)
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
else
throw;
}
if (block.rows() != 1 || res.in->read())
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
}
catch (const Exception & e)
{
if (e.code() == ErrorCodes::TOO_MANY_ROWS)
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
block = materializeBlock(block);
size_t columns = block.columns();
if (columns == 1)
scalar = block;
else
throw;
{
ColumnWithTypeAndName ctn;
ctn.type = std::make_shared<DataTypeTuple>(block.getDataTypes());
ctn.column = ColumnTuple::create(block.getColumns());
scalar.insert(ctn);
}
}
size_t columns = block.columns();
if (columns == 1)
const Settings & settings = data.context.getSettingsRef();
// Always convert to literals when there is no query context.
if (!settings.enable_scalar_subquery_optimization || worthConvertingToLiteral(scalar) || !data.context.hasQueryContext())
{
auto lit = std::make_unique<ASTLiteral>((*block.safeGetByPosition(0).column)[0]);
auto lit = std::make_unique<ASTLiteral>((*scalar.safeGetByPosition(0).column)[0]);
lit->alias = subquery.alias;
lit->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
ast = addTypeConversionToAST(std::move(lit), block.safeGetByPosition(0).type->getName());
ast = addTypeConversionToAST(std::move(lit), scalar.safeGetByPosition(0).type->getName());
}
else
{
auto tuple = std::make_shared<ASTFunction>();
tuple->alias = subquery.alias;
ast = tuple;
tuple->name = "tuple";
auto exp_list = std::make_shared<ASTExpressionList>();
tuple->arguments = exp_list;
tuple->children.push_back(tuple->arguments);
exp_list->children.resize(columns);
for (size_t i = 0; i < columns; ++i)
{
exp_list->children[i] = addTypeConversionToAST(
std::make_unique<ASTLiteral>((*block.safeGetByPosition(i).column)[0]),
block.safeGetByPosition(i).type->getName());
}
auto func = makeASTFunction("__getScalar", std::make_shared<ASTLiteral>(scalar_query_hash_str));
func->alias = subquery.alias;
func->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
ast = std::move(func);
}
data.scalars[scalar_query_hash_str] = std::move(scalar);
}
void ExecuteScalarSubqueriesMatcher::visit(const ASTFunction & func, ASTPtr & ast, Data & data)

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/typeid_cast.h>
#include <Core/Block.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
@ -36,6 +37,7 @@ public:
{
const Context & context;
size_t subquery_depth;
Scalars & scalars;
};
static bool needChildVisit(ASTPtr & node, const ASTPtr &);

View File

@ -54,7 +54,7 @@
#include <Parsers/queryToString.h>
#include <Interpreters/interpretSubquery.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/misc.h>
#include <Interpreters/ActionsVisitor.h>

View File

@ -1,17 +1,16 @@
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/Context.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
namespace DB
{
/// Must not acquire Context lock in constructor to avoid possibility of deadlocks.
ExternalDictionariesLoader::ExternalDictionariesLoader(
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_)
: ExternalLoader("external dictionary", &Logger::get("ExternalDictionariesLoader"))
, context(context_)
{
addConfigRepository(std::move(config_repository), {"dictionary", "name"});
enableAsyncLoading(true);
enablePeriodicUpdates(true);
}
@ -23,4 +22,21 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
return DictionaryFactory::instance().create(name, config, key_in_config, context);
}
void ExternalDictionariesLoader::addConfigRepository(
const std::string & repository_name, std::unique_ptr<IExternalLoaderConfigRepository> config_repository)
{
ExternalLoader::addConfigRepository(repository_name, std::move(config_repository), {"dictionary", "name"});
}
void ExternalDictionariesLoader::addDictionaryWithConfig(
const String & dictionary_name, const String & repo_name, const ASTCreateQuery & query, bool load_never_loading) const
{
ExternalLoader::addObjectAndLoad(
dictionary_name, /// names are equal
dictionary_name,
repo_name,
getDictionaryConfigurationFromAST(query),
"dictionary", load_never_loading);
}
}

View File

@ -4,6 +4,7 @@
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Interpreters/ExternalLoader.h>
#include <common/logger_useful.h>
#include <Parsers/ASTCreateQuery.h>
#include <memory>
@ -19,9 +20,7 @@ public:
using DictPtr = std::shared_ptr<const IDictionaryBase>;
/// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
ExternalDictionariesLoader(
ExternalLoaderConfigRepositoryPtr config_repository,
Context & context_);
ExternalDictionariesLoader(Context & context_);
DictPtr getDictionary(const std::string & name) const
{
@ -33,6 +32,18 @@ public:
return std::static_pointer_cast<const IDictionaryBase>(tryGetLoadable(name));
}
void addConfigRepository(
const std::string & repository_name,
std::unique_ptr<IExternalLoaderConfigRepository> config_repository);
/// Starts reloading of a specified object.
void addDictionaryWithConfig(
const String & dictionary_name,
const String & repo_name,
const ASTCreateQuery & query,
bool load_never_loading = false) const;
protected:
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
const std::string & key_in_config) const override;

View File

@ -20,12 +20,29 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
namespace
{
/// Lock mutex only in async mode
/// In other case does nothing
struct LoadingGuardForAsyncLoad
{
std::unique_lock<std::mutex> lock;
LoadingGuardForAsyncLoad(bool async, std::mutex & mutex)
{
if (async)
lock = std::unique_lock(mutex);
}
};
}
struct ExternalLoader::ObjectConfig
{
String config_path;
Poco::AutoPtr<Poco::Util::AbstractConfiguration> config;
String key_in_config;
String repository_name;
};
@ -40,26 +57,84 @@ public:
}
~LoadablesConfigReader() = default;
void addConfigRepository(std::unique_ptr<IExternalLoaderConfigRepository> repository, const ExternalLoaderConfigSettings & settings)
void addConfigRepository(
const String & name,
std::unique_ptr<IExternalLoaderConfigRepository> repository,
const ExternalLoaderConfigSettings & settings)
{
std::lock_guard lock{mutex};
repositories.emplace_back(std::move(repository), std::move(settings));
repositories.emplace(name, std::make_pair(std::move(repository), settings));
}
void removeConfigRepository(const String & name)
{
std::lock_guard lock{mutex};
repositories.erase(name);
}
using ObjectConfigsPtr = std::shared_ptr<const std::unordered_map<String /* object's name */, ObjectConfig>>;
/// Reads configuration files.
/// Reads configurations.
ObjectConfigsPtr read()
{
std::lock_guard lock{mutex};
std::lock_guard lock(mutex);
// Check last modification times of files and read those files which are new or changed.
if (!readLoadablesInfos())
return configs; // Nothing changed, so we can return the previous result.
return collectConfigs();
}
ObjectConfig updateLoadableInfo(
const String & external_name,
const String & object_name,
const String & repo_name,
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
const String & key)
{
std::lock_guard lock(mutex);
auto it = loadables_infos.find(object_name);
if (it == loadables_infos.end())
{
LoadablesInfos loadable_info;
loadables_infos[object_name] = loadable_info;
}
auto & loadable_info = loadables_infos[object_name];
ObjectConfig object_config{object_name, config, key, repo_name};
bool found = false;
for (auto iter = loadable_info.configs.begin(); iter != loadable_info.configs.end(); ++iter)
{
if (iter->first == external_name)
{
iter->second = object_config;
found = true;
break;
}
}
if (!found)
loadable_info.configs.emplace_back(external_name, object_config);
loadable_info.last_update_time = Poco::Timestamp{}; /// now
loadable_info.in_use = true;
return object_config;
}
private:
struct LoadablesInfos
{
Poco::Timestamp last_update_time = 0;
std::vector<std::pair<String, ObjectConfig>> configs; // Parsed loadable's contents.
bool in_use = true; // Whether the `LoadablesInfos` should be destroyed because the correspondent loadable is deleted.
};
/// Collect current configurations
ObjectConfigsPtr collectConfigs()
{
// Generate new result.
auto new_configs = std::make_shared<std::unordered_map<String /* object's name */, ObjectConfig>>();
for (const auto & [path, loadable_info] : loadables_infos)
for (const auto & [path, loadable_info] : loadables_infos)
{
for (const auto & [name, config] : loadable_info.configs)
{
@ -81,14 +156,6 @@ public:
return configs;
}
private:
struct LoadablesInfos
{
Poco::Timestamp last_update_time = 0;
std::vector<std::pair<String, ObjectConfig>> configs; // Parsed file's contents.
bool in_use = true; // Whether the ` LoadablesInfos` should be destroyed because the correspondent file is deleted.
};
/// Read files and store them to the map ` loadables_infos`.
bool readLoadablesInfos()
{
@ -100,58 +167,59 @@ private:
loadable_info.in_use = false;
}
for (const auto & [repository, settings] : repositories)
for (const auto & [repo_name, repo_with_settings] : repositories)
{
const auto names = repository->getAllLoadablesDefinitionNames();
for (const auto & name : names)
const auto names = repo_with_settings.first->getAllLoadablesDefinitionNames();
for (const auto & loadable_name : names)
{
auto it = loadables_infos.find(name);
if (it != loadables_infos.end())
auto it = loadables_infos.find(loadable_name);
if (it != loadables_infos.end())
{
LoadablesInfos & loadable_info = it->second;
if (readLoadablesInfo(*repository, name, settings, loadable_info))
if (readLoadablesInfo(repo_name, *repo_with_settings.first, loadable_name, repo_with_settings.second, loadable_info))
changed = true;
}
else
{
LoadablesInfos loadable_info;
if (readLoadablesInfo(*repository, name, settings, loadable_info))
if (readLoadablesInfo(repo_name, *repo_with_settings.first, loadable_name, repo_with_settings.second, loadable_info))
{
loadables_infos.emplace(name, std::move(loadable_info));
loadables_infos.emplace(loadable_name, std::move(loadable_info));
changed = true;
}
}
}
}
std::vector<String> deleted_files;
std::vector<String> deleted_names;
for (auto & [path, loadable_info] : loadables_infos)
if (!loadable_info.in_use)
deleted_files.emplace_back(path);
if (!deleted_files.empty())
deleted_names.emplace_back(path);
if (!deleted_names.empty())
{
for (const String & deleted_file : deleted_files)
loadables_infos.erase(deleted_file);
for (const String & deleted_name : deleted_names)
loadables_infos.erase(deleted_name);
changed = true;
}
return changed;
}
bool readLoadablesInfo(
const String & repo_name,
IExternalLoaderConfigRepository & repository,
const String & path,
const String & object_name,
const ExternalLoaderConfigSettings & settings,
LoadablesInfos & loadable_info) const
{
try
{
if (path.empty() || !repository.exists(path))
if (object_name.empty() || !repository.exists(object_name))
{
LOG_WARNING(log, "config file '" + path + "' does not exist");
LOG_WARNING(log, "Config file '" + object_name + "' does not exist");
return false;
}
auto update_time_from_repository = repository.getUpdateTime(path);
auto update_time_from_repository = repository.getUpdateTime(object_name);
/// Actually it can't be less, but for sure we check less or equal
if (update_time_from_repository <= loadable_info.last_update_time)
@ -160,31 +228,31 @@ private:
return false;
}
auto file_contents = repository.load(path);
auto file_contents = repository.load(object_name);
/// get all objects' definitions
Poco::Util::AbstractConfiguration::Keys keys;
file_contents->keys(keys);
/// for each object defined in xml config
/// for each object defined in repositories
std::vector<std::pair<String, ObjectConfig>> configs_from_file;
for (const auto & key : keys)
{
if (!startsWith(key, settings.external_config))
{
if (!startsWith(key, "comment") && !startsWith(key, "include_from"))
LOG_WARNING(log, path << ": file contains unknown node '" << key << "', expected '" << settings.external_config << "'");
LOG_WARNING(log, object_name << ": file contains unknown node '" << key << "', expected '" << settings.external_config << "'");
continue;
}
String name = file_contents->getString(key + "." + settings.external_name);
if (name.empty())
String external_name = file_contents->getString(key + "." + settings.external_name);
if (external_name.empty())
{
LOG_WARNING(log, path << ": node '" << key << "' defines " << type_name << " with an empty name. It's not allowed");
LOG_WARNING(log, object_name << ": node '" << key << "' defines " << type_name << " with an empty name. It's not allowed");
continue;
}
configs_from_file.emplace_back(name, ObjectConfig{path, file_contents, key});
configs_from_file.emplace_back(external_name, ObjectConfig{object_name, file_contents, key, repo_name});
}
loadable_info.configs = std::move(configs_from_file);
@ -194,16 +262,19 @@ private:
}
catch (...)
{
tryLogCurrentException(log, "Failed to read config file '" + path + "'");
tryLogCurrentException(log, "Failed to load config for dictionary '" + object_name + "'");
return false;
}
}
const String type_name;
Logger * log;
std::mutex mutex;
std::vector<std::pair<std::unique_ptr<IExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories;
using RepositoryPtr = std::unique_ptr<IExternalLoaderConfigRepository>;
using RepositoryWithSettings = std::pair<RepositoryPtr, ExternalLoaderConfigSettings>;
std::unordered_map<String, RepositoryWithSettings> repositories;
ObjectConfigsPtr configs;
std::unordered_map<String /* config path */, LoadablesInfos> loadables_infos;
};
@ -267,10 +338,10 @@ public:
else
{
const auto & new_config = new_config_it->second;
if (!isSameConfiguration(*info.config.config, info.config.key_in_config, *new_config.config, new_config.key_in_config))
if (!isSameConfiguration(*info.object_config.config, info.object_config.key_in_config, *new_config.config, new_config.key_in_config))
{
/// Configuration has been changed.
info.config = new_config;
info.object_config = new_config;
info.config_changed = true;
if (info.wasLoading())
@ -305,6 +376,12 @@ public:
event.notify_all();
}
void setSingleObjectConfigurationWithoutLoading(const String & external_name, const ObjectConfig & config)
{
std::lock_guard lock{mutex};
infos.emplace(external_name, Info{config});
}
/// Sets whether all the objects from the configuration should be always loaded (even if they aren't used).
void enableAlwaysLoadEverything(bool enable)
{
@ -326,7 +403,6 @@ public:
/// Sets whether the objects should be loaded asynchronously, each loading in a new thread (from the thread pool).
void enableAsyncLoading(bool enable)
{
std::lock_guard lock{mutex};
enable_async_loading = enable;
}
@ -432,8 +508,17 @@ public:
loaded_objects = collectLoadedObjects(filter_by_name);
}
/// Tries to finish loading of the objects for which the specified function returns true.
void load(const FilterByNameFunction & filter_by_name, LoadResults & loaded_results, Duration timeout = NO_TIMEOUT)
{
std::unique_lock lock{mutex};
loadImpl(filter_by_name, timeout, lock);
loaded_results = collectLoadResults(filter_by_name);
}
/// Tries to finish loading of all the objects during the timeout.
void load(Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) { load(allNames, loaded_objects, timeout); }
void load(LoadResults & loaded_results, Duration timeout = NO_TIMEOUT) { load(allNames, loaded_results, timeout); }
/// Starts reloading a specified object.
void reload(const String & name, bool load_never_loading = false)
@ -441,7 +526,9 @@ public:
std::lock_guard lock{mutex};
Info * info = getInfo(name);
if (!info)
{
return;
}
if (info->wasLoading() || load_never_loading)
{
@ -539,7 +626,7 @@ public:
private:
struct Info
{
Info(const ObjectConfig & config_) : config(config_) {}
Info(const ObjectConfig & object_config_) : object_config(object_config_) {}
bool loaded() const { return object != nullptr; }
bool failed() const { return !object && exception; }
@ -571,11 +658,12 @@ private:
result.exception = exception;
result.loading_start_time = loading_start_time;
result.loading_duration = loadingDuration();
result.origin = config.config_path;
result.origin = object_config.config_path;
result.repository_name = object_config.repository_name;
return result;
}
ObjectConfig config;
ObjectConfig object_config;
LoadablePtr object;
TimePoint loading_start_time;
TimePoint loading_end_time;
@ -618,8 +706,10 @@ private:
LoadResults load_results;
load_results.reserve(infos.size());
for (const auto & [name, info] : infos)
{
if (filter_by_name(name))
load_results.emplace_back(name, info.loadResult());
}
return load_results;
}
@ -691,48 +781,15 @@ private:
}
}
/// Does the loading, possibly in the separate thread.
void doLoading(const String & name, size_t loading_id, bool async)
/// Load one object, returns object ptr or exception
/// Do not require locking
std::pair<LoadablePtr, std::exception_ptr> loadOneObject(
const String & name,
const ObjectConfig & config,
bool config_changed,
LoadablePtr previous_version)
{
std::unique_lock<std::mutex> lock;
if (async)
{
setThreadName("ExterLdrJob");
lock = std::unique_lock{mutex}; /// If `async == false` the mutex is already locked.
}
SCOPE_EXIT({
if (async)
{
if (!lock.owns_lock())
lock.lock();
/// Remove the information about the thread after it finishes.
auto it = loading_ids.find(loading_id);
if (it != loading_ids.end())
{
it->second.detach();
loading_ids.erase(it);
}
}
});
/// We check here if this is exactly the same loading as we planned to perform.
/// This check is necessary because the object could be removed or load with another config before this thread even starts.
Info * info = getInfo(name);
if (!info || !info->loading() || (info->loading_id != loading_id))
return;
ObjectConfig config = info->config;
bool config_changed = info->config_changed;
LoadablePtr previous_version = info->object;
size_t error_count = info->error_count;
/// Use `create_function` to perform the actual loading.
/// It's much better to do it with `mutex` unlocked because the loading can take a lot of time
/// and require access to other objects.
if (async)
lock.unlock();
LoadablePtr new_object;
std::exception_ptr new_exception;
try
@ -743,10 +800,45 @@ private:
{
new_exception = std::current_exception();
}
return std::make_pair(new_object, new_exception);
if (!new_object && !new_exception)
throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR);
}
/// Return single object info, checks loading_id and name
std::optional<Info> getSingleObjectInfo(const String & name, size_t loading_id, bool async)
{
LoadingGuardForAsyncLoad lock(async, mutex);
Info * info = getInfo(name);
if (!info || !info->loading() || (info->loading_id != loading_id))
return {};
return *info;
}
/// Removes object loading_id from loading_ids if it present
/// in other case do nothin should by done with lock
void finishObjectLoading(size_t loading_id, const LoadingGuardForAsyncLoad &)
{
auto it = loading_ids.find(loading_id);
if (it != loading_ids.end())
{
it->second.detach();
loading_ids.erase(it);
}
}
/// Process loading result
/// Calculates next update time and process errors
void processLoadResult(
const String & name,
size_t loading_id,
LoadablePtr previous_version,
LoadablePtr new_object,
std::exception_ptr new_exception,
size_t error_count,
bool async)
{
LoadingGuardForAsyncLoad lock(async, mutex);
/// Calculate a new update time.
TimePoint next_update_time;
try
@ -755,7 +847,12 @@ private:
++error_count;
else
error_count = 0;
next_update_time = calculateNextUpdateTime(new_object, error_count);
LoadablePtr object = previous_version;
if (new_object)
object = new_object;
next_update_time = calculateNextUpdateTime(object, error_count);
}
catch (...)
{
@ -763,10 +860,8 @@ private:
next_update_time = TimePoint::max();
}
/// Lock the mutex again to store the changes.
if (async)
lock.lock();
info = getInfo(name);
Info * info = getInfo(name);
/// And again we should check if this is still the same loading as we were doing.
/// This is necessary because the object could be removed or load with another config while the `mutex` was unlocked.
@ -802,10 +897,38 @@ private:
if (new_object)
info->config_changed = false;
/// Notify `event` to recheck conditions in loadImpl() now.
if (async)
lock.unlock();
event.notify_all();
finishObjectLoading(loading_id, lock);
}
/// Does the loading, possibly in the separate thread.
void doLoading(const String & name, size_t loading_id, bool async)
{
try
{
/// We check here if this is exactly the same loading as we planned to perform.
/// This check is necessary because the object could be removed or load with another config before this thread even starts.
std::optional<Info> info = getSingleObjectInfo(name, loading_id, async);
if (!info)
return;
/// Use `create_function` to perform the actual loading.
/// It's much better to do it with `mutex` unlocked because the loading can take a lot of time
/// and require access to other objects.
auto [new_object, new_exception] = loadOneObject(name, info->object_config, info->config_changed, info->object);
if (!new_object && !new_exception)
throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR);
processLoadResult(name, loading_id, info->object, new_object, new_exception, info->error_count, async);
event.notify_all();
}
catch (...)
{
LoadingGuardForAsyncLoad lock(async, mutex);
finishObjectLoading(loading_id, lock);
throw;
}
}
void cancelLoading(const String & name)
@ -844,7 +967,8 @@ private:
TimePoint calculateNextUpdateTime(const LoadablePtr & loaded_object, size_t error_count) const
{
static constexpr auto never = TimePoint::max();
if (!error_count)
if (loaded_object)
{
if (!loaded_object->supportUpdates())
return never;
@ -854,8 +978,11 @@ private:
if (lifetime.min_sec == 0 || lifetime.max_sec == 0)
return never;
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
return std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
if (!error_count)
{
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
return std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
}
}
return std::chrono::system_clock::now() + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count));
@ -870,7 +997,7 @@ private:
ObjectConfigsPtr configs;
std::unordered_map<String, Info> infos;
bool always_load_everything = false;
bool enable_async_loading = false;
std::atomic<bool> enable_async_loading = false;
std::unordered_map<size_t, ThreadFromGlobalPool> loading_ids;
size_t next_loading_id = 1; /// should always be > 0
mutable pcg64 rnd_engine{randomSeed()};
@ -956,12 +1083,19 @@ ExternalLoader::ExternalLoader(const String & type_name_, Logger * log)
ExternalLoader::~ExternalLoader() = default;
void ExternalLoader::addConfigRepository(
std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings)
const std::string & repository_name,
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
const ExternalLoaderConfigSettings & config_settings)
{
config_files_reader->addConfigRepository(std::move(config_repository), config_settings);
config_files_reader->addConfigRepository(repository_name, std::move(config_repository), config_settings);
loading_dispatcher->setConfiguration(config_files_reader->read());
}
void ExternalLoader::removeConfigRepository(const std::string & repository_name)
{
config_files_reader->removeConfigRepository(repository_name);
}
void ExternalLoader::enableAlwaysLoadEverything(bool enable)
{
loading_dispatcher->enableAlwaysLoadEverything(enable);
@ -1035,23 +1169,52 @@ void ExternalLoader::load(const FilterByNameFunction & filter_by_name, Loadables
loading_dispatcher->load(loaded_objects, timeout);
}
void ExternalLoader::load(const FilterByNameFunction & filter_by_name, LoadResults & loaded_objects, Duration timeout) const
{
if (filter_by_name)
loading_dispatcher->load(filter_by_name, loaded_objects, timeout);
else
loading_dispatcher->load(loaded_objects, timeout);
}
void ExternalLoader::load(Loadables & loaded_objects, Duration timeout) const
{
return loading_dispatcher->load(loaded_objects, timeout);
}
void ExternalLoader::reload(const String & name, bool load_never_loading)
void ExternalLoader::reload(const String & name, bool load_never_loading) const
{
loading_dispatcher->setConfiguration(config_files_reader->read());
auto configs = config_files_reader->read();
loading_dispatcher->setConfiguration(configs);
loading_dispatcher->reload(name, load_never_loading);
}
void ExternalLoader::reload(bool load_never_loading)
void ExternalLoader::reload(bool load_never_loading) const
{
loading_dispatcher->setConfiguration(config_files_reader->read());
loading_dispatcher->reload(load_never_loading);
}
void ExternalLoader::addObjectAndLoad(
const String & name,
const String & external_name,
const String & repo_name,
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
const String & key,
bool load_never_loading) const
{
auto object_config = config_files_reader->updateLoadableInfo(external_name, name, repo_name, config, key);
loading_dispatcher->setSingleObjectConfigurationWithoutLoading(external_name, object_config);
LoadablePtr loaded_object;
if (load_never_loading)
loading_dispatcher->loadStrict(name, loaded_object);
else
loading_dispatcher->load(name, loaded_object, Duration::zero());
}
ExternalLoader::LoadablePtr ExternalLoader::createObject(
const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const
{

View File

@ -72,6 +72,7 @@ public:
TimePoint loading_start_time;
Duration loading_duration;
std::exception_ptr exception;
std::string repository_name;
};
using LoadResults = std::vector<std::pair<String, LoadResult>>;
@ -81,7 +82,12 @@ public:
/// Adds a repository which will be used to read configurations from.
void addConfigRepository(
std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings);
const std::string & repository_name,
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
const ExternalLoaderConfigSettings & config_settings);
/// Removes a repository which were used to read configurations.
void removeConfigRepository(const std::string & repository_name);
/// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
void enableAlwaysLoadEverything(bool enable);
@ -132,6 +138,7 @@ public:
/// Tries to finish loading of the objects for which the specified function returns true.
void load(const FilterByNameFunction & filter_by_name, Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
void load(const FilterByNameFunction & filter_by_name, LoadResults & load_results, Duration timeout = NO_TIMEOUT) const;
Loadables loadAndGet(const FilterByNameFunction & filter_by_name, Duration timeout = NO_TIMEOUT) const { Loadables loaded_objects; load(filter_by_name, loaded_objects, timeout); return loaded_objects; }
/// Tries to finish loading of all the objects during the timeout.
@ -140,16 +147,27 @@ public:
/// Starts reloading of a specified object.
/// `load_never_loading` specifies what to do if the object has never been loading before.
/// The function can either skip it (false) or load for the first time (true).
void reload(const String & name, bool load_never_loading = false);
/// Also function can load dictionary synchronously
void reload(const String & name, bool load_never_loading = false) const;
/// Starts reloading of all the objects.
/// `load_never_loading` specifies what to do with the objects which have never been loading before.
/// The function can either skip them (false) or load for the first time (true).
void reload(bool load_never_loading = false);
void reload(bool load_never_loading = false) const;
protected:
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0;
/// Reload object with already parsed configuration
void addObjectAndLoad(
const String & name, /// name of dictionary
const String & external_name, /// name of source (example xml-file, may contain more than dictionary)
const String & repo_name, /// name of repository (database name, or all xml files)
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
const String & key_in_config, /// key where we can start search of loadables (<dictionary>, <model>, etc)
bool load_never_loading = false) const;
private:
struct ObjectConfig;

View File

@ -0,0 +1,56 @@
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_DICTIONARY;
}
namespace
{
String trimDatabaseName(const std::string & loadable_definition_name, const DatabasePtr database)
{
const auto & dbname = database->getDatabaseName();
if (!startsWith(loadable_definition_name, dbname))
throw Exception(
"Loadable '" + loadable_definition_name + "' is not from database '" + database->getDatabaseName(), ErrorCodes::UNKNOWN_DICTIONARY);
/// dbname.loadable_name
///--> remove <---
return loadable_definition_name.substr(dbname.length() + 1);
}
}
LoadablesConfigurationPtr ExternalLoaderDatabaseConfigRepository::load(const std::string & loadable_definition_name) const
{
String dictname = trimDatabaseName(loadable_definition_name, database);
return getDictionaryConfigurationFromAST(database->getCreateDictionaryQuery(context, dictname)->as<const ASTCreateQuery &>());
}
bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable_definition_name) const
{
return database->isDictionaryExist(
context, trimDatabaseName(loadable_definition_name, database));
}
Poco::Timestamp ExternalLoaderDatabaseConfigRepository::getUpdateTime(const std::string & loadable_definition_name)
{
return database->getObjectMetadataModificationTime(context, trimDatabaseName(loadable_definition_name, database));
}
std::set<std::string> ExternalLoaderDatabaseConfigRepository::getAllLoadablesDefinitionNames() const
{
std::set<std::string> result;
const auto & dbname = database->getDatabaseName();
auto itr = database->getDictionariesIterator(context);
while (itr && itr->isValid())
{
result.insert(dbname + "." + itr->name());
itr->next();
}
return result;
}
}

View File

@ -0,0 +1,34 @@
#pragma once
#include <Interpreters/IExternalLoaderConfigRepository.h>
#include <Databases/IDatabase.h>
#include <Interpreters/Context.h>
namespace DB
{
/// Repository from database, which stores dictionary definitions on disk.
/// Tracks update time and existance of .sql files through IDatabase.
class ExternalLoaderDatabaseConfigRepository : public IExternalLoaderConfigRepository
{
public:
ExternalLoaderDatabaseConfigRepository(const DatabasePtr & database_, const Context & context_)
: database(database_)
, context(context_)
{
}
std::set<std::string> getAllLoadablesDefinitionNames() const override;
bool exists(const std::string & loadable_definition_name) const override;
Poco::Timestamp getUpdateTime(const std::string & loadable_definition_name) override;
LoadablesConfigurationPtr load(const std::string & loadable_definition_name) const override;
private:
DatabasePtr database;
Context context;
};
}

View File

@ -10,12 +10,10 @@ namespace ErrorCodes
}
ExternalModelsLoader::ExternalModelsLoader(
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
ExternalModelsLoader::ExternalModelsLoader(Context & context_)
: ExternalLoader("external model", &Logger::get("ExternalModelsLoader"))
, context(context_)
{
addConfigRepository(std::move(config_repository), {"model", "name"});
enablePeriodicUpdates(true);
}
@ -40,4 +38,8 @@ std::shared_ptr<const IExternalLoadable> ExternalModelsLoader::create(
}
}
void ExternalModelsLoader::addConfigRepository(const String & name, std::unique_ptr<IExternalLoaderConfigRepository> config_repository)
{
ExternalLoader::addConfigRepository(name, std::move(config_repository), {"models", "name"});
}
}

View File

@ -18,15 +18,17 @@ public:
using ModelPtr = std::shared_ptr<const IModel>;
/// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
ExternalModelsLoader(
ExternalLoaderConfigRepositoryPtr config_repository,
Context & context_);
ExternalModelsLoader(Context & context_);
ModelPtr getModel(const std::string & name) const
{
return std::static_pointer_cast<const IModel>(getLoadable(name));
}
void addConfigRepository(const String & name,
std::unique_ptr<IExternalLoaderConfigRepository> config_repository);
protected:
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
const std::string & key_in_config) const override;

View File

@ -28,17 +28,9 @@ void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &)
}
else
{
// FIXME: make a better matcher using `names`?
size_t best_match = 0;
for (const auto & table_names : tables)
{
if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table_names.first))
if (match > best_match)
{
best_match = match;
best_table = &table_names.first;
}
}
size_t best_table_pos = 0;
if (IdentifierSemantic::chooseTable(identifier, tables, best_table_pos))
best_table = &tables[best_table_pos].first;
}
identifier_table.emplace_back(&identifier, best_table);

View File

@ -24,6 +24,11 @@ UInt64 calculateDurationWithBackoff(pcg64 & rnd_engine, size_t error_count)
if (error_count < 1)
error_count = 1;
/// max seconds is 600 and 2 ** 10 == 1024
if (error_count > 11)
error_count = 11;
std::uniform_int_distribution<UInt64> distribution(0, static_cast<UInt64>(std::exp2(error_count - 1)));
return std::min<UInt64>(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine));
}

View File

@ -5,6 +5,61 @@
namespace DB
{
namespace ErrorCodes
{
extern const int AMBIGUOUS_COLUMN_NAME;
}
namespace
{
const DatabaseAndTableWithAlias & extractTable(const DatabaseAndTableWithAlias & table)
{
return table;
}
const DatabaseAndTableWithAlias & extractTable(const TableWithColumnNames & table)
{
return table.first;
}
template <typename T>
IdentifierSemantic::ColumnMatch tryChooseTable(const ASTIdentifier & identifier, const std::vector<T> & tables,
size_t & best_table_pos, bool allow_ambiguous)
{
using ColumnMatch = IdentifierSemantic::ColumnMatch;
best_table_pos = 0;
auto best_match = ColumnMatch::NoMatch;
size_t same_match = 0;
for (size_t i = 0; i < tables.size(); ++i)
{
auto match = IdentifierSemantic::canReferColumnToTable(identifier, extractTable(tables[i]));
if (match != ColumnMatch::NoMatch)
{
if (match > best_match)
{
best_match = match;
best_table_pos = i;
same_match = 0;
}
else if (match == best_match)
++same_match;
}
}
if ((best_match != ColumnMatch::NoMatch) && same_match)
{
if (!allow_ambiguous)
throw Exception("Ambiguous column '" + identifier.name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
return ColumnMatch::Ambiguous;
}
return best_match;
}
}
std::optional<String> IdentifierSemantic::getColumnName(const ASTIdentifier & node)
{
if (!node.semantic->special)
@ -37,26 +92,36 @@ std::optional<String> IdentifierSemantic::getTableName(const ASTPtr & ast)
return {};
}
void IdentifierSemantic::setNeedLongName(ASTIdentifier & identifier, bool value)
{
identifier.semantic->need_long_name = value;
}
bool IdentifierSemantic::canBeAlias(const ASTIdentifier & identifier)
{
return identifier.semantic->can_be_alias;
}
void IdentifierSemantic::setMembership(ASTIdentifier & identifier, size_t table_no)
void IdentifierSemantic::setMembership(ASTIdentifier & identifier, size_t table_pos)
{
identifier.semantic->membership = table_no;
identifier.semantic->membership = table_pos;
identifier.semantic->can_be_alias = false;
}
size_t IdentifierSemantic::getMembership(const ASTIdentifier & identifier)
std::optional<size_t> IdentifierSemantic::getMembership(const ASTIdentifier & identifier)
{
return identifier.semantic->membership;
}
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<DatabaseAndTableWithAlias> & tables,
size_t & best_table_pos, bool ambiguous)
{
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
return tryChooseTable<DatabaseAndTableWithAlias>(identifier, tables, best_table_pos, ambiguous) != no_match;
}
bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector<TableWithColumnNames> & tables,
size_t & best_table_pos, bool ambiguous)
{
static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch;
return tryChooseTable<TableWithColumnNames>(identifier, tables, best_table_pos, ambiguous) != no_match;
}
std::pair<String, String> IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
{
if (identifier.name_parts.size() > 2)
@ -84,24 +149,49 @@ bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier
return false;
}
size_t IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier,
const DatabaseAndTableWithAlias & db_and_table)
{
/// database.table.column
if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table))
return 2;
return ColumnMatch::DbAndTable;
/// table.column or alias.column.
if (doesIdentifierBelongTo(identifier, db_and_table.table) ||
doesIdentifierBelongTo(identifier, db_and_table.alias))
return 1;
/// alias.column
if (doesIdentifierBelongTo(identifier, db_and_table.alias))
return ColumnMatch::TableAlias;
return 0;
/// table.column
if (doesIdentifierBelongTo(identifier, db_and_table.table))
{
if (!db_and_table.alias.empty())
return ColumnMatch::AliasedTableName;
else
return ColumnMatch::TableName;
}
return ColumnMatch::NoMatch;
}
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t to_strip)
/// Strip qualificators from left side of column name.
/// Example: 'database.table.name' -> 'name'.
void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
auto match = IdentifierSemantic::canReferColumnToTable(identifier, db_and_table);
size_t to_strip = 0;
switch (match)
{
case ColumnMatch::TableName:
case ColumnMatch::AliasedTableName:
case ColumnMatch::TableAlias:
to_strip = 1;
break;
case ColumnMatch::DbAndTable:
to_strip = 2;
break;
default:
break;
}
if (!to_strip)
return;
@ -117,18 +207,6 @@ void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t t
identifier.name.swap(new_name);
}
void IdentifierSemantic::setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
size_t match = IdentifierSemantic::canReferColumnToTable(identifier, db_and_table);
setColumnShortName(identifier, match);
if (match)
identifier.semantic->can_be_alias = false;
if (identifier.semantic->need_long_name)
setColumnLongName(identifier, db_and_table);
}
void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
String prefix = db_and_table.getQualifiedNamePrefix();
@ -141,16 +219,4 @@ void IdentifierSemantic::setColumnLongName(ASTIdentifier & identifier, const Dat
}
}
String IdentifierSemantic::columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
ASTPtr copy = identifier.clone();
setColumnNormalName(copy->as<ASTIdentifier &>(), db_and_table);
return copy->getAliasOrColumnName();
}
String IdentifierSemantic::columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table)
{
return db_and_table.getQualifiedNamePrefix() + identifier.shortName();
}
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <optional>
#include <Parsers/ASTIdentifier.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
@ -9,14 +11,23 @@ namespace DB
struct IdentifierSemanticImpl
{
bool special = false; /// for now it's 'not a column': tables, subselects and some special stuff like FORMAT
bool need_long_name = false;/// if column presents in multiple tables we need qualified names
bool can_be_alias = true; /// if it's a cropped name it could not be an alias
size_t membership = 0; /// table position in join (starting from 1) detected by qualifier or 0 if not detected.
std::optional<size_t> membership; /// table position in join
};
/// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier
struct IdentifierSemantic
{
enum class ColumnMatch
{
NoMatch,
AliasedTableName, /// column qualified with table name (but table has an alias so its priority is lower than TableName)
TableName, /// column qualified with table name
DbAndTable, /// column qualified with database and table name
TableAlias, /// column qualified with table alias
Ambiguous,
};
/// @returns name for column identifiers
static std::optional<String> getColumnName(const ASTIdentifier & node);
static std::optional<String> getColumnName(const ASTPtr & ast);
@ -26,20 +37,20 @@ struct IdentifierSemantic
static std::optional<String> getTableName(const ASTPtr & ast);
static std::pair<String, String> extractDatabaseAndTable(const ASTIdentifier & identifier);
static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static String columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setColumnLongName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
static void setNeedLongName(ASTIdentifier & identifier, bool); /// if set setColumnNormalName makes qualified name
static bool canBeAlias(const ASTIdentifier & identifier);
static void setMembership(ASTIdentifier & identifier, size_t table_no);
static size_t getMembership(const ASTIdentifier & identifier);
static void setMembership(ASTIdentifier &, size_t table_no);
static std::optional<size_t> getMembership(const ASTIdentifier & identifier);
static bool chooseTable(const ASTIdentifier &, const std::vector<DatabaseAndTableWithAlias> & tables, size_t & best_table_pos,
bool ambiguous = false);
static bool chooseTable(const ASTIdentifier &, const std::vector<TableWithColumnNames> & tables, size_t & best_table_pos,
bool ambiguous = false);
private:
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table);
static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table);
static void setColumnShortName(ASTIdentifier & identifier, size_t match);
};
}

View File

@ -69,6 +69,7 @@ namespace ErrorCodes
extern const int THERE_IS_NO_DEFAULT_VALUE;
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE;
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
extern const int DICTIONARY_ALREADY_EXISTS;
}
@ -163,7 +164,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
if (need_write_metadata)
Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path);
database->loadTables(context, has_force_restore_data_flag);
database->loadStoredObjects(context, has_force_restore_data_flag);
}
catch (...)
{
@ -630,6 +631,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
if (database->isTableExist(context, table_name))
{
/// TODO Check structure of table
if (create.if_not_exists)
return {};
else if (create.replace_view)
@ -704,6 +706,39 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
return {};
}
BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
{
String dictionary_name = create.table;
String database_name = !create.database.empty() ? create.database : context.getCurrentDatabase();
auto guard = context.getDDLGuard(database_name, dictionary_name);
DatabasePtr database = context.getDatabase(database_name);
if (database->isDictionaryExist(context, dictionary_name))
{
/// TODO Check structure of dictionary
if (create.if_not_exists)
return {};
else
throw Exception(
"Dictionary " + database_name + "." + dictionary_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
}
if (create.attach)
{
auto query = context.getCreateDictionaryQuery(database_name, dictionary_name);
create = query->as<ASTCreateQuery &>();
create.attach = true;
}
if (create.attach)
database->attachDictionary(dictionary_name, context);
else
database->createDictionary(context, dictionary_name, query_ptr);
return {};
}
BlockIO InterpreterCreateQuery::execute()
{
@ -713,11 +748,11 @@ BlockIO InterpreterCreateQuery::execute()
/// CREATE|ATTACH DATABASE
if (!create.database.empty() && create.table.empty())
{
return createDatabase(create);
}
else
else if (!create.is_dictionary)
return createTable(create);
else
return createDictionary(create);
}
@ -742,13 +777,22 @@ void InterpreterCreateQuery::checkAccess(const ASTCreateQuery & create)
throw Exception("Cannot create database. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
}
String object = "table";
if (create.is_dictionary)
{
if (readonly)
throw Exception("Cannot create dictionary in readonly mode", ErrorCodes::READONLY);
object = "dictionary";
}
if (create.temporary && readonly >= 2)
return;
if (readonly)
throw Exception("Cannot create table in readonly mode", ErrorCodes::READONLY);
throw Exception("Cannot create table or dictionary in readonly mode", ErrorCodes::READONLY);
throw Exception("Cannot create table. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
throw Exception("Cannot create " + object + ". DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
}
}

View File

@ -51,6 +51,7 @@ public:
private:
BlockIO createDatabase(ASTCreateQuery & create);
BlockIO createTable(ASTCreateQuery & create);
BlockIO createDictionary(ASTCreateQuery & create);
/// Calculate list of columns, constraints, indices, etc... of table and return columns.
ColumnsDescription setProperties(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const;

View File

@ -4,6 +4,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Parsers/ASTDropQuery.h>
#include <Storages/IStorage.h>
#include <Common/escapeForFileName.h>
@ -24,6 +25,7 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR;
extern const int UNKNOWN_TABLE;
extern const int QUERY_IS_PROHIBITED;
extern const int UNKNOWN_DICTIONARY;
}
@ -40,15 +42,26 @@ BlockIO InterpreterDropQuery::execute()
return executeDDLQueryOnCluster(query_ptr, context, {drop.database});
if (!drop.table.empty())
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
{
if (!drop.is_dictionary)
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
else
return executeToDictionary(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
}
else if (!drop.database.empty())
return executeToDatabase(drop.database, drop.kind, drop.if_exists);
else
throw Exception("Database and table names is empty.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Nothing to drop, both names are empty.", ErrorCodes::LOGICAL_ERROR);
}
BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock)
BlockIO InterpreterDropQuery::executeToTable(
String & database_name_,
String & table_name,
ASTDropQuery::Kind kind,
bool if_exists,
bool if_temporary,
bool no_ddl_lock)
{
if (if_temporary || database_name_.empty())
{
@ -135,6 +148,50 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t
return {};
}
BlockIO InterpreterDropQuery::executeToDictionary(
String & database_name_,
String & dictionary_name,
ASTDropQuery::Kind kind,
bool if_exists,
bool is_temporary,
bool no_ddl_lock)
{
if (is_temporary)
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
String database_name = database_name_.empty() ? context.getCurrentDatabase() : database_name_;
auto ddl_guard = (!no_ddl_lock ? context.getDDLGuard(database_name, dictionary_name) : nullptr);
DatabasePtr database = tryGetDatabase(database_name, false);
if (!database || !database->isDictionaryExist(context, dictionary_name))
{
if (!if_exists)
throw Exception(
"Dictionary " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(dictionary_name) + " doesn't exist.",
ErrorCodes::UNKNOWN_DICTIONARY);
else
return {};
}
if (kind == ASTDropQuery::Kind::Detach)
{
/// Drop dictionary from memory, don't touch data and metadata
database->detachDictionary(dictionary_name, context);
}
else if (kind == ASTDropQuery::Kind::Truncate)
{
throw Exception("Cannot TRUNCATE dictionary", ErrorCodes::SYNTAX_ERROR);
}
else if (kind == ASTDropQuery::Kind::Drop)
{
database->removeDictionary(context, dictionary_name);
}
return {};
}
BlockIO InterpreterDropQuery::executeToTemporaryTable(String & table_name, ASTDropQuery::Kind kind)
{
if (kind == ASTDropQuery::Kind::Detach)
@ -185,12 +242,18 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ
}
else if (kind == ASTDropQuery::Kind::Drop)
{
for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next())
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
{
String current_table_name = iterator->name();
executeToTable(database_name, current_table_name, kind, false, false, false);
}
for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next())
{
String current_dictionary = iterator->name();
executeToDictionary(database_name, current_dictionary, kind, false, false, false);
}
auto context_lock = context.getLock();
/// Someone could have time to delete the database before us.

View File

@ -32,6 +32,8 @@ private:
BlockIO executeToTable(String & database_name, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock);
BlockIO executeToDictionary(String & database_name, String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool if_temporary, bool no_ddl_lock);
DatabasePtr tryGetDatabase(String & database_name, bool exists);
DatabaseAndTable tryGetDatabaseAndTable(String & database_name, String & table_name, bool if_exists);

View File

@ -13,6 +13,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
BlockIO InterpreterExistsQuery::execute()
{
BlockIO res;
@ -32,11 +37,24 @@ Block InterpreterExistsQuery::getSampleBlock()
BlockInputStreamPtr InterpreterExistsQuery::executeImpl()
{
const auto & ast = query_ptr->as<ASTExistsTableQuery &>();
bool res = ast.temporary ? context.isExternalTableExist(ast.table) : context.isTableExist(ast.database, ast.table);
ASTQueryWithTableAndOutput * exists_query;
bool result = false;
if (exists_query = query_ptr->as<ASTExistsTableQuery>(); exists_query)
{
if (exists_query->temporary)
result = context.isExternalTableExist(exists_query->table);
else
result = context.isTableExist(exists_query->database, exists_query->table);
}
else if (exists_query = query_ptr->as<ASTExistsDictionaryQuery>(); exists_query)
{
if (exists_query->temporary)
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
result = context.isDictionaryExists(exists_query->database, exists_query->table);
}
return std::make_shared<OneBlockInputStream>(Block{{
ColumnUInt8::create(1, res),
ColumnUInt8::create(1, result),
std::make_shared<DataTypeUInt8>(),
"result" }});
}

View File

@ -137,6 +137,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
{
return std::make_unique<InterpreterExistsQuery>(query, context);
}
else if (query->as<ASTExistsDictionaryQuery>())
{
return std::make_unique<InterpreterExistsQuery>(query, context);
}
else if (query->as<ASTShowCreateTableQuery>())
{
return std::make_unique<InterpreterShowCreateQuery>(query, context);
@ -145,6 +149,10 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
{
return std::make_unique<InterpreterShowCreateQuery>(query, context);
}
else if (query->as<ASTShowCreateDictionaryQuery>())
{
return std::make_unique<InterpreterShowCreateQuery>(query, context);
}
else if (query->as<ASTDescribeQuery>())
{
return std::make_unique<InterpreterDescribeQuery>(query, context);

View File

@ -305,6 +305,12 @@ InterpreterSelectQuery::InterpreterSelectQuery(
syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze(
query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList());
/// Save scalar sub queries's results in the query context
if (context.hasQueryContext())
for (const auto & it : syntax_analyzer_result->getScalars())
context.getQueryContext().addScalar(it.first, it.second);
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
query_ptr, syntax_analyzer_result, context,
NameSet(required_result_column_names.begin(), required_result_column_names.end()),

View File

@ -42,22 +42,30 @@ Block InterpreterShowCreateQuery::getSampleBlock()
BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
{
/// FIXME: try to prettify this cast using `as<>()`
const auto & ast = dynamic_cast<const ASTQueryWithTableAndOutput &>(*query_ptr);
if (ast.temporary && !ast.database.empty())
throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR);
ASTPtr create_query;
if (ast.temporary)
create_query = context.getCreateExternalTableQuery(ast.table);
else if (ast.table.empty())
create_query = context.getCreateDatabaseQuery(ast.database);
else
create_query = context.getCreateTableQuery(ast.database, ast.table);
ASTQueryWithTableAndOutput * show_query;
if (show_query = query_ptr->as<ASTShowCreateTableQuery>(); show_query)
{
if (show_query->temporary)
create_query = context.getCreateExternalTableQuery(show_query->table);
else
create_query = context.getCreateTableQuery(show_query->database, show_query->table);
}
else if (show_query = query_ptr->as<ASTShowCreateDatabaseQuery>(); show_query)
{
if (show_query->temporary)
throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR);
create_query = context.getCreateDatabaseQuery(show_query->database);
}
else if (show_query = query_ptr->as<ASTShowCreateDictionaryQuery>(); show_query)
{
if (show_query->temporary)
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
create_query = context.getCreateDictionaryQuery(show_query->database, show_query->table);
}
if (!create_query && ast.temporary)
throw Exception("Unable to show the create query of " + ast.table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
if (!create_query && show_query->temporary)
throw Exception("Unable to show the create query of " + show_query->table + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
std::stringstream stream;
formatAST(*create_query, stream, false, true);

View File

@ -44,10 +44,21 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
context.assertDatabaseExists(database, false);
std::stringstream rewritten_query;
rewritten_query << "SELECT name FROM system.tables WHERE ";
rewritten_query << "SELECT name FROM system.";
if (query.dictionaries)
rewritten_query << "dictionaries ";
else
rewritten_query << "tables ";
rewritten_query << "WHERE ";
if (query.temporary)
{
if (query.dictionaries)
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
rewritten_query << "is_temporary";
}
else
rewritten_query << "database = " << std::quoted(database, '\'');

Some files were not shown because too many files have changed in this diff Show More