diff --git a/.github/labeler.keywords.yml b/.github/labeler.keywords.yml deleted file mode 100644 index c70ea45de53..00000000000 --- a/.github/labeler.keywords.yml +++ /dev/null @@ -1 +0,0 @@ -pr-feature: "New Feature" diff --git a/.github/labeler.yml b/.github/labeler.yml deleted file mode 100644 index 72cf714f039..00000000000 --- a/.github/labeler.yml +++ /dev/null @@ -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/**/*" diff --git a/.github/stale.yml b/.github/stale.yml index 6b18b043853..6628bbbd305 100644 --- a/.github/stale.yml +++ b/.github/stale.yml @@ -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: diff --git a/.github/workflows/labeler.yml b/.github/workflows/labeler.yml deleted file mode 100644 index 0110ef7b516..00000000000 --- a/.github/workflows/labeler.yml +++ /dev/null @@ -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 }}" diff --git a/README.md b/README.md index 83cf3e9adbc..08be0b9ed07 100644 --- a/README.md +++ b/README.md @@ -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. + diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 06c33fb7e74..b0a271b21ac 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -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) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 78307c15123..c70f3d8a068 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -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) diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 195f8c01270..c0b2eccfd29 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -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); }); diff --git a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp index b188c0bea88..594cddfd3db 100644 --- a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -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: diff --git a/dbms/programs/server/ReplicasStatusHandler.cpp b/dbms/programs/server/ReplicasStatusHandler.cpp index cbbdacccc56..22e059d9ccc 100644 --- a/dbms/programs/server/ReplicasStatusHandler.cpp +++ b/dbms/programs/server/ReplicasStatusHandler.cpp @@ -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(table.get()); diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 5385057fd03..e274dca6bdc 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -37,10 +37,12 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include @@ -920,6 +922,12 @@ int Server::main(const std::vector & /*args*/) global_context->tryCreateEmbeddedDictionaries(); global_context->getExternalDictionariesLoader().enableAlwaysLoadEverything(true); } + + auto dictionaries_repository = std::make_unique(config(), "dictionaries_config"); + global_context->getExternalDictionariesLoader().addConfigRepository("", std::move(dictionaries_repository)); + + auto models_repository = std::make_unique(config(), "models_config"); + global_context->getExternalModelsLoader().addConfigRepository("", std::move(models_repository)); } catch (...) { diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 83749975f34..88b1eb6ae3e 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -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 diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index 834d66bba28..561ed4d0eca 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -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 getReadTimeouts(const Settings & global_settings); diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index c8d33922167..6e9bb527c97 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -180,7 +180,21 @@ 9000 - + + + + + 127.0.0.1 + 9000 + + + + + 127.0.0.2 + 9000 + + + diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index a6e533d8dd2..95dff73f870 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -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(*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(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(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()) diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index 03a771c257f..8b507a4172a 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -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); diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index 5c05ee9c5f5..d7934924242 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -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 & data) { std::lock_guard lock(cancel_mutex); diff --git a/dbms/src/Client/MultiplexedConnections.h b/dbms/src/Client/MultiplexedConnections.h index b8567dcd979..b26c9569422 100644 --- a/dbms/src/Client/MultiplexedConnections.h +++ b/dbms/src/Client/MultiplexedConnections.h @@ -27,6 +27,8 @@ public: std::vector && 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 & data); diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index 6bd99fb8f01..5ad4a281f80 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -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.") \ diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index cfa89af96d4..7abc7dc9232 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -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; diff --git a/dbms/src/Common/XDBCBridgeHelper.h b/dbms/src/Common/XDBCBridgeHelper.h index 9b4cb96b468..c8ecb30ec68 100644 --- a/dbms/src/Common/XDBCBridgeHelper.h +++ b/dbms/src/Common/XDBCBridgeHelper.h @@ -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); diff --git a/dbms/src/Core/Protocol.h b/dbms/src/Core/Protocol.h index b50d018f9ce..1992234b3fe 100644 --- a/dbms/src/Core/Protocol.h +++ b/dbms/src/Core/Protocol.h @@ -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) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 30752113a6b..9361b909590 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -379,6 +379,8 @@ struct Settings : public SettingsCollection 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") \ diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 9e9d47f9516..61432939a95 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -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 && 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(); } diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.h b/dbms/src/DataStreams/RemoteBlockInputStream.h index af8d79c324c..89f4e84f080 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.h +++ b/dbms/src/DataStreams/RemoteBlockInputStream.h @@ -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 && 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; diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index aecc1b9125f..52dde4efe9b 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -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(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(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(); +} + + +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(listTables(context, filter_by_name)); + return std::make_unique(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 &) { diff --git a/dbms/src/Databases/DatabaseDictionary.h b/dbms/src/Databases/DatabaseDictionary.h index 1e1af7ef581..9e88b50a68e 100644 --- a/dbms/src/Databases/DatabaseDictionary.h +++ b/dbms/src/Databases/DatabaseDictionary.h @@ -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: diff --git a/dbms/src/Databases/DatabaseFactory.cpp b/dbms/src/Databases/DatabaseFactory.cpp index fe81bd3d495..a968e9140c6 100644 --- a/dbms/src/Databases/DatabaseFactory.cpp +++ b/dbms/src/Databases/DatabaseFactory.cpp @@ -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()->value.safeGet(); const auto & mysql_database_name = arguments[1]->as()->value.safeGet(); @@ -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()->value.safeGet(); return std::make_shared(database_name, metadata_path, cache_expiration_time_seconds, context); diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index 35a28e539bc..d84ee61aea5 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -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(); +} + +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(), 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); diff --git a/dbms/src/Databases/DatabaseLazy.h b/dbms/src/Databases/DatabaseLazy.h index 649116508b3..c268f58945c 100644 --- a/dbms/src/Databases/DatabaseLazy.h +++ b/dbms/src/Databases/DatabaseLazy.h @@ -2,11 +2,13 @@ #include #include +#include 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; }; - } diff --git a/dbms/src/Databases/DatabaseMemory.cpp b/dbms/src/Databases/DatabaseMemory.cpp index 1356a28d245..0badc9b4df4 100644 --- a/dbms/src/Databases/DatabaseMemory.cpp +++ b/dbms/src/Databases/DatabaseMemory.cpp @@ -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(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 { diff --git a/dbms/src/Databases/DatabaseMemory.h b/dbms/src/Databases/DatabaseMemory.h index 33bb8787168..45f51a177f7 100644 --- a/dbms/src/Databases/DatabaseMemory.h +++ b/dbms/src/Databases/DatabaseMemory.h @@ -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; diff --git a/dbms/src/Databases/DatabaseMySQL.cpp b/dbms/src/Databases/DatabaseMySQL.cpp index 9fd16cc94a8..8e0e092b1d8 100644 --- a/dbms/src/Databases/DatabaseMySQL.cpp +++ b/dbms/src/Databases/DatabaseMySQL.cpp @@ -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 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(tables); + return std::make_unique(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 lock(mutex); diff --git a/dbms/src/Databases/DatabaseMySQL.h b/dbms/src/Databases/DatabaseMySQL.h index 7044a594b4c..3ca8722b248 100644 --- a/dbms/src/Databases/DatabaseMySQL.h +++ b/dbms/src/Databases/DatabaseMySQL.h @@ -6,6 +6,7 @@ #include #include #include +#include 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(); + } 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 { diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 941935f37f4..cb5921d74b9 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -1,37 +1,49 @@ #include -#include -#include -#include #include #include #include #include +#include +#include +#include +#include #include -#include #include +#include +#include #include +#include +#include +#include #include #include + + 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 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(); + 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(); + + 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(), !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(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 { diff --git a/dbms/src/Databases/DatabaseOnDisk.h b/dbms/src/Databases/DatabaseOnDisk.h index 231db6fdccb..49910f67e11 100644 --- a/dbms/src/Databases/DatabaseOnDisk.h +++ b/dbms/src/Databases/DatabaseOnDisk.h @@ -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 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 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; - 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().table = to_table_name; diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 5987597e3e9..a50ad4615e4 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -11,10 +11,17 @@ #include #include #include +#include +#include #include #include +#include +#include +#include #include +#include #include +#include #include #include @@ -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; - 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; + 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(); + 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 tables_processed {0}; + std::atomic tables_processed{0}; + std::atomic 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()); }); } pool.wait(); /// After all tables was basically initialized, startup them. startupTables(pool); + + /// Add database as repository + auto dictionaries_repository = std::make_unique(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 tables_processed {0}; + std::atomic 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); } } diff --git a/dbms/src/Databases/DatabaseOrdinary.h b/dbms/src/Databases/DatabaseOrdinary.h index d8f7e1b3612..8046b7482b8 100644 --- a/dbms/src/Databases/DatabaseOrdinary.h +++ b/dbms/src/Databases/DatabaseOrdinary.h @@ -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; }; } diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 1a278fbb03d..2feda6fc2b3 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -1,15 +1,20 @@ #include +#include +#include #include #include #include #include +#include #include #include #include +#include #include #include #include +#include #include @@ -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(); - /// 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 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(); - 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(); - 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(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(tables); + return std::make_unique(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(std::move(filtered_tables)); + + return std::make_unique(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(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(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() diff --git a/dbms/src/Databases/DatabasesCommon.h b/dbms/src/Databases/DatabasesCommon.h index 9eba0fc0596..f07136aa927 100644 --- a/dbms/src/Databases/DatabasesCommon.h +++ b/dbms/src/Databases/DatabasesCommon.h @@ -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 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_)) { } }; diff --git a/dbms/src/Databases/IDatabase.h b/dbms/src/Databases/IDatabase.h index 8fce43c4b2d..ac1ccc096d5 100644 --- a/dbms/src/Databases/IDatabase.h +++ b/dbms/src/Databases/IDatabase.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -19,16 +20,14 @@ struct ConstraintsDescription; class ColumnsDescription; struct IndicesDescription; struct TableStructureWriteLockHolder; +using Dictionaries = std::set; 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; +/// 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; +using DatabaseDictionariesIteratorPtr = std::unique_ptr; /** 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; diff --git a/dbms/src/Dictionaries/CMakeLists.txt b/dbms/src/Dictionaries/CMakeLists.txt index 51ec9289ae6..027aebc9653 100644 --- a/dbms/src/Dictionaries/CMakeLists.txt +++ b/dbms/src/Dictionaries/CMakeLists.txt @@ -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}) diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index c3a78150f05..57498221141 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -611,7 +611,7 @@ void registerDictionaryCache(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); }; - factory.registerLayout("cache", create_layout); + factory.registerLayout("cache", create_layout, false); } diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index a3b4e8c5cfb..6586f979687 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -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(dict_struct, config, config_prefix + ".clickhouse", sample_block, context); }; diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.h b/dbms/src/Dictionaries/ClickHouseDictionarySource.h index 3df962708bd..84bbd78b93a 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.h +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.h @@ -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); diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp index 3478e631076..b27adc20636 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -415,7 +415,7 @@ void registerDictionaryComplexKeyCache(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; return std::make_unique(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); } diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index 5e08ce3295e..41a5caaa768 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -755,7 +755,7 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory) const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); return std::make_unique(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); } diff --git a/dbms/src/Dictionaries/DictionaryFactory.cpp b/dbms/src/Dictionaries/DictionaryFactory.cpp index 43ae9d5623b..52703ce36d1 100644 --- a/dbms/src/Dictionaries/DictionaryFactory.cpp +++ b/dbms/src/Dictionaries/DictionaryFactory.cpp @@ -3,6 +3,7 @@ #include #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; diff --git a/dbms/src/Dictionaries/DictionaryFactory.h b/dbms/src/Dictionaries/DictionaryFactory.h index dbfdc563aa4..d9efd3f42f1 100644 --- a/dbms/src/Dictionaries/DictionaryFactory.h +++ b/dbms/src/Dictionaries/DictionaryFactory.h @@ -1,6 +1,7 @@ #pragma once #include "IDictionary.h" +#include 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; - 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; LayoutRegistry registered_layouts; + using LayoutComplexity = std::unordered_map; + LayoutComplexity layout_complexity; }; } diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.cpp b/dbms/src/Dictionaries/DictionarySourceFactory.cpp index 01e7440d4e9..b9ddf9b961b 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.cpp +++ b/dbms/src/Dictionaries/DictionarySourceFactory.cpp @@ -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); diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.h b/dbms/src/Dictionaries/DictionarySourceFactory.h index 11b7a1f55a8..e6c75f6d075 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.h +++ b/dbms/src/Dictionaries/DictionarySourceFactory.h @@ -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; diff --git a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp index d76de3abe0e..68c48d2b457 100644 --- a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp @@ -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}; diff --git a/dbms/src/Dictionaries/FileDictionarySource.cpp b/dbms/src/Dictionaries/FileDictionarySource.cpp index 1505c2629f6..1c9f9d60ce3 100644 --- a/dbms/src/Dictionaries/FileDictionarySource.cpp +++ b/dbms/src/Dictionaries/FileDictionarySource.cpp @@ -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}; diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index d1c6a138c89..68afdd355b8 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -724,7 +724,7 @@ void registerDictionaryFlat(DictionaryFactory & factory) const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; - factory.registerLayout("flat", create_layout); + factory.registerLayout("flat", create_layout, false); } diff --git a/dbms/src/Dictionaries/HTTPDictionarySource.cpp b/dbms/src/Dictionaries/HTTPDictionarySource.cpp index 184470ae7c5..2a8269d4047 100644 --- a/dbms/src/Dictionaries/HTTPDictionarySource.cpp +++ b/dbms/src/Dictionaries/HTTPDictionarySource.cpp @@ -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}; diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index d81b259b184..1c6fd602ba9 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -787,8 +787,8 @@ void registerDictionaryHashed(DictionaryFactory & factory) const bool sparse = name == "sparse_hashed"; return std::make_unique(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); } } diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.cpp b/dbms/src/Dictionaries/RangeHashedDictionary.cpp index 3fb29d747c0..ae67027c210 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.cpp +++ b/dbms/src/Dictionaries/RangeHashedDictionary.cpp @@ -691,7 +691,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; - factory.registerLayout("range_hashed", create_layout); + factory.registerLayout("range_hashed", create_layout, false); } } diff --git a/dbms/src/Dictionaries/TrieDictionary.cpp b/dbms/src/Dictionaries/TrieDictionary.cpp index 7a5adee63b6..f7f4f8c33a6 100644 --- a/dbms/src/Dictionaries/TrieDictionary.cpp +++ b/dbms/src/Dictionaries/TrieDictionary.cpp @@ -767,7 +767,7 @@ void registerDictionaryTrie(DictionaryFactory & factory) // This is specialised trie for storing IPv4 and IPv6 prefixes. return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; - factory.registerLayout("ip_trie", create_layout); + factory.registerLayout("ip_trie", create_layout, true); } } diff --git a/dbms/src/Dictionaries/XDBCDictionarySource.cpp b/dbms/src/Dictionaries/XDBCDictionarySource.cpp index 627092844ec..12b6b72fceb 100644 --- a/dbms/src/Dictionaries/XDBCDictionarySource.cpp +++ b/dbms/src/Dictionaries/XDBCDictionarySource.cpp @@ -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>( context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); diff --git a/dbms/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/dbms/src/Dictionaries/getDictionaryConfigurationFromAST.cpp new file mode 100644 index 00000000000..67d4e99cac6 --- /dev/null +++ b/dbms/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -0,0 +1,444 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +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 + * + * 10 + * 100 + * + */ +void buildLifetimeConfiguration( + AutoPtr doc, + AutoPtr root, + const ASTDictionaryLifetime * lifetime) +{ + + AutoPtr lifetime_element(doc->createElement("lifetime")); + AutoPtr min_element(doc->createElement("min")); + AutoPtr max_element(doc->createElement("max")); + AutoPtr min_sec(doc->createTextNode(toString(lifetime->min_sec))); + min_element->appendChild(min_sec); + AutoPtr 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 + * + * + * + * + * And next definition + * LAYOUT(CACHE(SIZE_IN_CELLS 1000)) + * to the next one + * + * + * 1000 + * + * + */ +void buildLayoutConfiguration( + AutoPtr doc, + AutoPtr root, + const ASTDictionaryLayout * layout) +{ + AutoPtr layout_element(doc->createElement("layout")); + root->appendChild(layout_element); + AutoPtr 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 layout_type_parameter_element(doc->createElement(param->first)); + const ASTLiteral & literal = param->second->as(); + AutoPtr value(doc->createTextNode(toString(literal.value.get()))); + 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 + * StartDate + * EndDate + */ +void buildRangeConfiguration(AutoPtr doc, AutoPtr root, const ASTDictionaryRange * range) +{ + // appends value to root + auto appendElem = [&doc, &root](const std::string & key, const std::string & value) + { + AutoPtr element(doc->createElement(key)); + AutoPtr name(doc->createElement("name")); + AutoPtr 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(); + result.push_back(key_part->name); + } + return result; +} + +/** + * Transofrms single dictionary attribute to configuration + * third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77 + * to + * + * third_column + * UInt8 + * 2 + * (rand() % 100) * 77 + * + */ +void buildSingleAttribute( + AutoPtr doc, + AutoPtr root, + const ASTDictionaryAttributeDeclaration * dict_attr) +{ + AutoPtr attribute_element(doc->createElement("attribute")); + root->appendChild(attribute_element); + + AutoPtr name_element(doc->createElement("name")); + AutoPtr name(doc->createTextNode(dict_attr->name)); + name_element->appendChild(name); + attribute_element->appendChild(name_element); + + AutoPtr type_element(doc->createElement("type")); + AutoPtr type(doc->createTextNode(queryToString(dict_attr->type))); + type_element->appendChild(type); + attribute_element->appendChild(type_element); + + AutoPtr 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 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 expression_element(doc->createElement("expression")); + AutoPtr expression(doc->createTextNode(queryToString(dict_attr->expression))); + expression_element->appendChild(expression); + attribute_element->appendChild(expression_element); + } + + if (dict_attr->hierarchical) + { + AutoPtr hierarchical_element(doc->createElement("hierarchical")); + AutoPtr hierarchical(doc->createTextNode("true")); + hierarchical_element->appendChild(hierarchical); + attribute_element->appendChild(hierarchical_element); + } + + if (dict_attr->injective) + { + AutoPtr injective_element(doc->createElement("injective")); + AutoPtr injective(doc->createTextNode("true")); + injective_element->appendChild(injective); + attribute_element->appendChild(injective_element); + } + + if (dict_attr->is_object_id) + { + AutoPtr is_object_id_element(doc->createElement("is_object_id")); + AutoPtr 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 + * Attr1 + * or + * + * + * Attr1 + * UInt8 + * + * ... + * fe + * + * + */ +void buildPrimaryKeyConfiguration( + AutoPtr doc, + AutoPtr 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 id_element(doc->createElement("id")); + root->appendChild(id_element); + AutoPtr name_element(doc->createElement("name")); + id_element->appendChild(name_element); + AutoPtr 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 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(); + 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 buildDictionaryAttributesConfiguration( + AutoPtr doc, + AutoPtr root, + const ASTExpressionList * dictionary_attributes, + const Names & key_columns) +{ + const auto & children = dictionary_attributes->children; + std::unordered_set dictionary_attributes_names; + for (size_t i = 0; i < children.size(); ++i) + { + const ASTDictionaryAttributeDeclaration * dict_attr = children[i]->as(); + 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 doc, + AutoPtr 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(); + AutoPtr current_xml_element(doc->createElement(pair->first)); + root->appendChild(current_xml_element); + + if (auto identifier = pair->second->as(); identifier) + { + AutoPtr value(doc->createTextNode(identifier->name)); + current_xml_element->appendChild(value); + } + else if (auto literal = pair->second->as(); literal) + { + AutoPtr value(doc->createTextNode(getUnescapedFieldString(literal->value))); + current_xml_element->appendChild(value); + } + else if (auto list = pair->second->as(); 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 + * + * + * localhost + * ... + * + * 127.0.0.1 + * ... + * + * + * + */ +void buildSourceConfiguration(AutoPtr doc, AutoPtr root, const ASTFunctionWithKeyValueArguments * source) +{ + AutoPtr outer_element(doc->createElement("source")); + root->appendChild(outer_element); + AutoPtr source_element(doc->createElement(source->name)); + outer_element->appendChild(source_element); + buildConfigurationFromFunctionWithKeyValueArguments(doc, source_element, source->elements->as()); +} + +/** 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 & 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 xml_document(new Poco::XML::Document()); + AutoPtr document_root(xml_document->createElement("dictionaries")); + xml_document->appendChild(document_root); + AutoPtr current_dictionary(xml_document->createElement("dictionary")); + document_root->appendChild(current_dictionary); + AutoPtr conf(new Poco::Util::XMLConfiguration()); + + AutoPtr name_element(xml_document->createElement("name")); + current_dictionary->appendChild(name_element); + AutoPtr name(xml_document->createTextNode(query.database + "." + query.table)); + name_element->appendChild(name); + + AutoPtr 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; +} + +} diff --git a/dbms/src/Dictionaries/getDictionaryConfigurationFromAST.h b/dbms/src/Dictionaries/getDictionaryConfigurationFromAST.h new file mode 100644 index 00000000000..bb48765c492 --- /dev/null +++ b/dbms/src/Dictionaries/getDictionaryConfigurationFromAST.h @@ -0,0 +1,15 @@ +#pragma once + +#include +#include + +namespace DB +{ +using DictionaryConfigurationPtr = Poco::AutoPtr; + +/// 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); + +} diff --git a/dbms/src/Dictionaries/tests/gtest_dictionary_configuration.cpp b/dbms/src/Dictionaries/tests/gtest_dictionary_configuration.cpp new file mode 100644 index 00000000000..7cda09a259a --- /dev/null +++ b/dbms/src/Dictionaries/tests/gtest_dictionary_configuration.cpp @@ -0,0 +1,224 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +using namespace DB; + +static bool registered = false; +/// For debug +std::string configurationToString(const DictionaryConfigurationPtr & config) +{ + const Poco::Util::XMLConfiguration * xml_config = dynamic_cast(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(); + 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(); + 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(); + 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(); + 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); +} diff --git a/dbms/src/Functions/CRC.cpp b/dbms/src/Functions/CRC.cpp new file mode 100644 index 00000000000..e506812d94c --- /dev/null +++ b/dbms/src/Functions/CRC.cpp @@ -0,0 +1,146 @@ +#include +#include +#include +#include + +namespace +{ + +template +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 +struct CRCImpl +{ + using ReturnType = T; + + static T make_crc(const unsigned char *buf, size_t size) + { + static CRCBase 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 +{ + static constexpr auto name = "CRC64"; +}; + +static constexpr UInt32 CRC32_IEEE = 0xedb88320; +struct CRC32IEEEImpl : public CRCImpl +{ + 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 +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 & 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 & 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 & /*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(&buf[0]) + offset; + return Impl::make_crc(p, size); + } +}; + +template +using FunctionCRC = FunctionStringOrArrayToT, 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; +// Uses CRC-32-IEEE 802.3 polynomial +using FunctionCRC32IEEE = FunctionCRC; +// Uses CRC-64-ECMA polynomial +using FunctionCRC64ECMA = FunctionCRC; + +template +void registerFunctionCRCImpl(FunctionFactory & factory) +{ + factory.registerFunction(T::name, FunctionFactory::CaseInsensitive); +} + +void registerFunctionCRC(FunctionFactory & factory) +{ + registerFunctionCRCImpl(factory); + registerFunctionCRCImpl(factory); + registerFunctionCRCImpl(factory); +} + +} diff --git a/dbms/src/Functions/CRC32.cpp b/dbms/src/Functions/CRC32.cpp deleted file mode 100644 index 80e0f163571..00000000000 --- a/dbms/src/Functions/CRC32.cpp +++ /dev/null @@ -1,68 +0,0 @@ -#include -#include -#include -#include - - -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 & 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 & 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 & /*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(&buf[0]) + offset; - return crc32(0L, p, size); - } -}; - -struct NameCRC32 -{ - static constexpr auto name = "CRC32"; -}; -using FunctionCRC32 = FunctionStringOrArrayToT; - -void registerFunctionCRC32(FunctionFactory & factory) -{ - factory.registerFunction(NameCRC32::name, FunctionFactory::CaseInsensitive); -} - -} diff --git a/dbms/src/Functions/getScalar.cpp b/dbms/src/Functions/getScalar.cpp new file mode 100644 index 00000000000..b04fcdc83f5 --- /dev/null +++ b/dbms/src/Functions/getScalar.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +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(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(*arguments[0].column).getField().get(); + 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(); +} + +} diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index 4c8a5ec84f7..6c3aeacbd1d 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -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); diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp index 1f4219b18f2..df407750d35 100644 --- a/dbms/src/Functions/registerFunctionsString.cpp +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -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); diff --git a/dbms/src/Interpreters/ActionLocksManager.cpp b/dbms/src/Interpreters/ActionLocksManager.cpp index 79578252325..df0907f551b 100644 --- a/dbms/src/Interpreters/ActionLocksManager.cpp +++ b/dbms/src/Interpreters/ActionLocksManager.cpp @@ -23,7 +23,7 @@ template 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()); } diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index c587d1826e1..3b3f1ddde63 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -29,7 +29,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index 7cb137e3f68..4fc1988bbef 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -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(); diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 905827205b4..3c141b56152 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -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(shard_info.pool, query, header, context, nullptr, throttler, external_tables, processed_stage); + auto stream = std::make_shared( + 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( - std::move(connections), query, header, context, nullptr, throttler, external_tables, stage); + std::move(connections), query, header, context, nullptr, throttler, scalars, external_tables, stage); } }; diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 20bac52d393..c3a55f5348f 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -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; }; diff --git a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp index f8938f2a7d3..894e1ea3a5a 100644 --- a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -164,7 +164,8 @@ size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vector #include #include +#include #include #include #include @@ -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(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(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 cmd) +void Context::addXDBCBridgeCommand(std::unique_ptr cmd) const { auto lock = getLock(); shared->bridge_commands.emplace_back(std::move(cmd)); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index bba4fdb18e9..b135d16e4e7 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -105,6 +105,9 @@ using InputInitializer = std::function; /// Callback for reading blocks of data from client for function input() using InputBlocksReader = std::function; +/// Scalar results of sub queries +using Scalars = std::map; + /// 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 cmd); + void addXDBCBridgeCommand(std::unique_ptr cmd) const; IHostContextPtr & getHostContext(); const IHostContextPtr & getHostContext() const; diff --git a/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp b/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp index 94b38b2c991..4fd67c2031b 100644 --- a/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -4,7 +4,7 @@ #include #include #include -#include // for functionIsInOperator +#include #include #include #include @@ -90,9 +90,12 @@ public: using TypeToVisit = const ASTFunction; CheckExpressionVisitorData(const std::vector & 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 & tables; + const std::vector & joined_tables; + std::vector tables; std::map> 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 left_best{0, 0}; - std::pair 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; } diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 59f7f46be70..ebd42b09afd 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -6,14 +6,17 @@ #include #include -#include +#include #include #include #include #include +#include #include +#include +#include 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 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(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(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(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((*block.safeGetByPosition(0).column)[0]); + auto lit = std::make_unique((*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(); - tuple->alias = subquery.alias; - ast = tuple; - tuple->name = "tuple"; - auto exp_list = std::make_shared(); - 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((*block.safeGetByPosition(i).column)[0]), - block.safeGetByPosition(i).type->getName()); - } + auto func = makeASTFunction("__getScalar", std::make_shared(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) diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h index 85d68a54110..6613cafd495 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include namespace DB @@ -36,6 +37,7 @@ public: { const Context & context; size_t subquery_depth; + Scalars & scalars; }; static bool needChildVisit(ASTPtr & node, const ASTPtr &); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 14849763ef3..f694f74989a 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -54,7 +54,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/src/Interpreters/ExternalDictionariesLoader.cpp b/dbms/src/Interpreters/ExternalDictionariesLoader.cpp index 5e84da0a0cd..601e42c56e4 100644 --- a/dbms/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/dbms/src/Interpreters/ExternalDictionariesLoader.cpp @@ -1,17 +1,16 @@ #include #include #include +#include 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 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); +} } diff --git a/dbms/src/Interpreters/ExternalDictionariesLoader.h b/dbms/src/Interpreters/ExternalDictionariesLoader.h index 04916f90d9d..ae2ffc8bcc8 100644 --- a/dbms/src/Interpreters/ExternalDictionariesLoader.h +++ b/dbms/src/Interpreters/ExternalDictionariesLoader.h @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -19,9 +20,7 @@ public: using DictPtr = std::shared_ptr; /// 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(tryGetLoadable(name)); } + void addConfigRepository( + const std::string & repository_name, + std::unique_ptr 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; diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index b4c61f5a5da..6486b394623 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -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 lock; + LoadingGuardForAsyncLoad(bool async, std::mutex & mutex) + { + if (async) + lock = std::unique_lock(mutex); + } +}; + +} struct ExternalLoader::ObjectConfig { String config_path; Poco::AutoPtr config; String key_in_config; + String repository_name; }; @@ -40,26 +57,84 @@ public: } ~LoadablesConfigReader() = default; - void addConfigRepository(std::unique_ptr repository, const ExternalLoaderConfigSettings & settings) + void addConfigRepository( + const String & name, + std::unique_ptr 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>; - /// 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 & 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> 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>(); - 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> 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 deleted_files; + std::vector 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> 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, ExternalLoaderConfigSettings>> repositories; + using RepositoryPtr = std::unique_ptr; + using RepositoryWithSettings = std::pair; + std::unordered_map repositories; ObjectConfigsPtr configs; std::unordered_map 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 loadOneObject( + const String & name, + const ObjectConfig & config, + bool config_changed, + LoadablePtr previous_version) { - std::unique_lock 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 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 = 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 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 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 infos; bool always_load_everything = false; - bool enable_async_loading = false; + std::atomic enable_async_loading = false; std::unordered_map 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 config_repository, const ExternalLoaderConfigSettings & config_settings) + const std::string & repository_name, + std::unique_ptr 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 & 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 { diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index 5c1fd1c0416..1ae5efdb4b4 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -72,6 +72,7 @@ public: TimePoint loading_start_time; Duration loading_duration; std::exception_ptr exception; + std::string repository_name; }; using LoadResults = std::vector>; @@ -81,7 +82,12 @@ public: /// Adds a repository which will be used to read configurations from. void addConfigRepository( - std::unique_ptr config_repository, const ExternalLoaderConfigSettings & config_settings); + const std::string & repository_name, + std::unique_ptr 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 & config, + const String & key_in_config, /// key where we can start search of loadables (, , etc) + bool load_never_loading = false) const; + private: struct ObjectConfig; diff --git a/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp new file mode 100644 index 00000000000..bd89f27def1 --- /dev/null +++ b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.cpp @@ -0,0 +1,56 @@ +#include +#include + +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()); +} + +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 ExternalLoaderDatabaseConfigRepository::getAllLoadablesDefinitionNames() const +{ + std::set result; + const auto & dbname = database->getDatabaseName(); + auto itr = database->getDictionariesIterator(context); + while (itr && itr->isValid()) + { + result.insert(dbname + "." + itr->name()); + itr->next(); + } + return result; +} + +} diff --git a/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h new file mode 100644 index 00000000000..343ed8cf038 --- /dev/null +++ b/dbms/src/Interpreters/ExternalLoaderDatabaseConfigRepository.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include + +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 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; +}; + +} diff --git a/dbms/src/Interpreters/ExternalModelsLoader.cpp b/dbms/src/Interpreters/ExternalModelsLoader.cpp index 462e8110249..6bdf8341906 100644 --- a/dbms/src/Interpreters/ExternalModelsLoader.cpp +++ b/dbms/src/Interpreters/ExternalModelsLoader.cpp @@ -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 ExternalModelsLoader::create( } } +void ExternalModelsLoader::addConfigRepository(const String & name, std::unique_ptr config_repository) +{ + ExternalLoader::addConfigRepository(name, std::move(config_repository), {"models", "name"}); +} } diff --git a/dbms/src/Interpreters/ExternalModelsLoader.h b/dbms/src/Interpreters/ExternalModelsLoader.h index fa860d08b2b..0e95828bd77 100644 --- a/dbms/src/Interpreters/ExternalModelsLoader.h +++ b/dbms/src/Interpreters/ExternalModelsLoader.h @@ -18,15 +18,17 @@ public: using ModelPtr = std::shared_ptr; /// 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(getLoadable(name)); } + void addConfigRepository(const String & name, + std::unique_ptr config_repository); + + protected: LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config) const override; diff --git a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp b/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp index 8173ce3256a..daf9ca57fb9 100644 --- a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp +++ b/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp @@ -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); diff --git a/dbms/src/Interpreters/IExternalLoadable.cpp b/dbms/src/Interpreters/IExternalLoadable.cpp index 18439cf999f..5c2df092179 100644 --- a/dbms/src/Interpreters/IExternalLoadable.cpp +++ b/dbms/src/Interpreters/IExternalLoadable.cpp @@ -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 distribution(0, static_cast(std::exp2(error_count - 1))); return std::min(backoff_max_sec, backoff_initial_sec + distribution(rnd_engine)); } diff --git a/dbms/src/Interpreters/IdentifierSemantic.cpp b/dbms/src/Interpreters/IdentifierSemantic.cpp index 361462c0d1d..34910ef039f 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.cpp +++ b/dbms/src/Interpreters/IdentifierSemantic.cpp @@ -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 +IdentifierSemantic::ColumnMatch tryChooseTable(const ASTIdentifier & identifier, const std::vector & 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 IdentifierSemantic::getColumnName(const ASTIdentifier & node) { if (!node.semantic->special) @@ -37,26 +92,36 @@ std::optional 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 IdentifierSemantic::getMembership(const ASTIdentifier & identifier) { return identifier.semantic->membership; } +bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector & tables, + size_t & best_table_pos, bool ambiguous) +{ + static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch; + return tryChooseTable(identifier, tables, best_table_pos, ambiguous) != no_match; +} + +bool IdentifierSemantic::chooseTable(const ASTIdentifier & identifier, const std::vector & tables, + size_t & best_table_pos, bool ambiguous) +{ + static constexpr auto no_match = IdentifierSemantic::ColumnMatch::NoMatch; + return tryChooseTable(identifier, tables, best_table_pos, ambiguous) != no_match; +} + std::pair 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(), db_and_table); - return copy->getAliasOrColumnName(); -} - -String IdentifierSemantic::columnLongName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table) -{ - return db_and_table.getQualifiedNamePrefix() + identifier.shortName(); -} - } diff --git a/dbms/src/Interpreters/IdentifierSemantic.h b/dbms/src/Interpreters/IdentifierSemantic.h index b4bf87e7fef..82b5ff31dde 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.h +++ b/dbms/src/Interpreters/IdentifierSemantic.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include @@ -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 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 getColumnName(const ASTIdentifier & node); static std::optional getColumnName(const ASTPtr & ast); @@ -26,20 +37,20 @@ struct IdentifierSemantic static std::optional getTableName(const ASTPtr & ast); static std::pair 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 getMembership(const ASTIdentifier & identifier); + static bool chooseTable(const ASTIdentifier &, const std::vector & tables, size_t & best_table_pos, + bool ambiguous = false); + static bool chooseTable(const ASTIdentifier &, const std::vector & 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); }; } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 8913f6a4b12..573c655035d 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -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(); + 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); } + } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index 2472f0321e1..fe32f44b84e 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -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; diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 565863d139a..3b1d80ee35f 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -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. diff --git a/dbms/src/Interpreters/InterpreterDropQuery.h b/dbms/src/Interpreters/InterpreterDropQuery.h index 8ca91610cbb..114f2af5f85 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.h +++ b/dbms/src/Interpreters/InterpreterDropQuery.h @@ -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); diff --git a/dbms/src/Interpreters/InterpreterExistsQuery.cpp b/dbms/src/Interpreters/InterpreterExistsQuery.cpp index 57682cbe30b..3d3b41a1818 100644 --- a/dbms/src/Interpreters/InterpreterExistsQuery.cpp +++ b/dbms/src/Interpreters/InterpreterExistsQuery.cpp @@ -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(); - 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(); 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(); 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(Block{{ - ColumnUInt8::create(1, res), + ColumnUInt8::create(1, result), std::make_shared(), "result" }}); } diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 6c9bd314b88..d27c9c8baeb 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -137,6 +137,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); @@ -145,6 +149,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 905fe6e3f04..dc7331f7031 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -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( query_ptr, syntax_analyzer_result, context, NameSet(required_result_column_names.begin(), required_result_column_names.end()), diff --git a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp index 74299ffaf4a..1bc253c8aaf 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -42,22 +42,30 @@ Block InterpreterShowCreateQuery::getSampleBlock() BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() { - /// FIXME: try to prettify this cast using `as<>()` - const auto & ast = dynamic_cast(*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(); 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(); 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(); 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); diff --git a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp index dcfe76adb82..f4fd8b77036 100644 --- a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -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, '\''); diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 6da0b9333ac..664efca90f9 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -315,7 +315,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) DatabasePtr & database = elem.second; const String & database_name = elem.first; - for (auto iterator = database->getIterator(system_context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(system_context); iterator->isValid(); iterator->next()) { if (dynamic_cast(iterator->table().get())) replica_names.emplace_back(database_name, iterator->name()); diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 2a10a7a28ae..fba978df9c2 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -147,7 +147,13 @@ struct ColumnAliasesMatcher auto it = rev_aliases.find(long_name); if (it == rev_aliases.end()) { - bool last_table = IdentifierSemantic::canReferColumnToTable(*identifier, tables.back()); + bool last_table = false; + { + size_t best_table_pos = 0; + if (IdentifierSemantic::chooseTable(*identifier, tables, best_table_pos)) + last_table = (best_table_pos + 1 == tables.size()); + } + if (!last_table) { String alias = hide_prefix + long_name; @@ -202,17 +208,15 @@ struct ColumnAliasesMatcher bool last_table = false; String long_name; - for (auto & table : data.tables) + + size_t table_pos = 0; + if (IdentifierSemantic::chooseTable(node, data.tables, table_pos)) { - if (IdentifierSemantic::canReferColumnToTable(node, table)) - { - if (!long_name.empty()) - throw Exception("Cannot refer column '" + node.name + "' to one table", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name - long_name = node.name; - if (&table == &data.tables.back()) - last_table = true; - } + auto & table = data.tables[table_pos]; + IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name + long_name = node.name; + if (&table == &data.tables.back()) + last_table = true; } if (long_name.empty()) diff --git a/dbms/src/Interpreters/MarkTableIdentifiersVisitor.cpp b/dbms/src/Interpreters/MarkTableIdentifiersVisitor.cpp new file mode 100644 index 00000000000..f110e0ba2df --- /dev/null +++ b/dbms/src/Interpreters/MarkTableIdentifiersVisitor.cpp @@ -0,0 +1,47 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +bool MarkTableIdentifiersMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child) +{ + if (child->as()) + return false; + if (node->as()) + return false; + return true; +} + +void MarkTableIdentifiersMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * node_func = ast->as()) + visit(*node_func, ast, data); + else if (auto * node_table = ast->as()) + visit(*node_table, ast, data); +} + +void MarkTableIdentifiersMatcher::visit(ASTTableExpression & table, ASTPtr &, Data &) +{ + if (table.database_and_table_name) + setIdentifierSpecial(table.database_and_table_name); +} + +void MarkTableIdentifiersMatcher::visit(const ASTFunction & func, ASTPtr &, Data & data) +{ + /// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`. + if (functionIsInOrGlobalInOperator(func.name)) + { + auto & ast = func.arguments->children.at(1); + if (auto opt_name = tryGetIdentifierName(ast)) + if (!data.aliases.count(*opt_name)) + setIdentifierSpecial(ast); + } +} + +} diff --git a/dbms/src/Interpreters/MarkTableIdentifiersVisitor.h b/dbms/src/Interpreters/MarkTableIdentifiersVisitor.h new file mode 100644 index 00000000000..f882f322bcf --- /dev/null +++ b/dbms/src/Interpreters/MarkTableIdentifiersVisitor.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ASTFunction; +struct ASTTableExpression; + +class MarkTableIdentifiersMatcher +{ +public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + const Aliases & aliases; + }; + + static bool needChildVisit(ASTPtr & node, const ASTPtr & child); + static void visit(ASTPtr & ast, Data & data); + +private: + static void visit(ASTTableExpression & table, ASTPtr &, Data &); + static void visit(const ASTFunction & func, ASTPtr &, Data &); +}; + +using MarkTableIdentifiersVisitor = MarkTableIdentifiersMatcher::Visitor; + +} diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 2a307c6ed7f..27772b8fc94 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -412,6 +413,9 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast QueryAliasesVisitor::Data query_aliases_data{aliases}; QueryAliasesVisitor(query_aliases_data).visit(ast); + MarkTableIdentifiersVisitor::Data mark_tables_data{aliases}; + MarkTableIdentifiersVisitor(mark_tables_data).visit(ast); + QueryNormalizer::Data normalizer_data(aliases, settings); QueryNormalizer(normalizer_data).visit(ast); diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index e109e4a63fd..9d6d28a68f6 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include namespace DB @@ -63,34 +62,6 @@ private: }; -void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data) -{ - auto & aliases = data.aliases; - String & func_name = node.name; - ASTPtr & func_arguments = node.arguments; - - /// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`. - if (functionIsInOrGlobalInOperator(func_name)) - { - auto & ast = func_arguments->children.at(1); - if (auto opt_name = tryGetIdentifierName(ast)) - if (!aliases.count(*opt_name)) - setIdentifierSpecial(ast); - } - - /// Special cases for count function. - String func_name_lowercase = Poco::toLower(func_name); - if (startsWith(func_name_lowercase, "count")) - { - /// Select implementation of countDistinct based on settings. - /// Important that it is done as query rewrite. It means rewritten query - /// will be sent to remote servers during distributed query execution, - /// and on all remote servers, function implementation will be same. - if (endsWith(func_name, "Distinct") && func_name_lowercase == "countdistinct") - func_name = data.settings.count_distinct_implementation; - } -} - void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) { auto & current_asts = data.current_asts; @@ -144,16 +115,8 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) } } -/// mark table identifiers as 'not columns' void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data) { - /// mark table Identifiers as 'not a column' - if (node.table_expression) - { - auto & expr = node.table_expression->as(); - setIdentifierSpecial(expr.database_and_table_name); - } - /// normalize JOIN ON section if (node.table_join) { @@ -177,7 +140,6 @@ void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data if (needVisitChild(child)) visit(child, data); -#if 1 /// TODO: legacy? /// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children, /// but also in where_expression and having_expression. if (select.prewhere()) @@ -186,7 +148,6 @@ void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr &, Data & data visit(select.refWhere(), data); if (select.having()) visit(select.refHaving(), data); -#endif } /// Don't go into subqueries. @@ -243,9 +204,7 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data) data.current_alias = my_alias; } - if (auto * node_func = ast->as()) - visit(*node_func, ast, data); - else if (auto * node_id = ast->as()) + if (auto * node_id = ast->as()) visit(*node_id, ast, data); else if (auto * node_tables = ast->as()) visit(*node_tables, ast, data); diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 6d6fea86e44..b842ae3f018 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -2,25 +2,13 @@ #include -#include #include #include namespace DB { -inline bool functionIsInOperator(const String & name) -{ - return name == "in" || name == "notIn"; -} - -inline bool functionIsInOrGlobalInOperator(const String & name) -{ - return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn"; -} - class ASTSelectQuery; -class ASTFunction; class ASTIdentifier; struct ASTTablesInSelectQueryElement; class Context; @@ -33,13 +21,11 @@ class QueryNormalizer { const UInt64 max_ast_depth; const UInt64 max_expanded_ast_elements; - const String count_distinct_implementation; template ExtractedSettings(const T & settings) : max_ast_depth(settings.max_ast_depth), - max_expanded_ast_elements(settings.max_expanded_ast_elements), - count_distinct_implementation(settings.count_distinct_implementation) + max_expanded_ast_elements(settings.max_expanded_ast_elements) {} }; @@ -80,7 +66,6 @@ private: static void visit(ASTPtr & query, Data & data); static void visit(ASTIdentifier &, ASTPtr &, Data &); - static void visit(ASTFunction &, const ASTPtr &, Data &); static void visit(ASTTablesInSelectQueryElement &, const ASTPtr &, Data &); static void visit(ASTSelectQuery &, const ASTPtr &, Data &); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 228aea0b2f2..67a1b3ea7db 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -72,6 +73,26 @@ namespace using LogAST = DebugASTLog; /// set to true to enable logs +/// Select implementation of countDistinct based on settings. +/// Important that it is done as query rewrite. It means rewritten query +/// will be sent to remote servers during distributed query execution, +/// and on all remote servers, function implementation will be same. +struct CustomizeFunctionsData +{ + using TypeToVisit = ASTFunction; + + const String & count_distinct; + + void visit(ASTFunction & func, ASTPtr &) + { + if (Poco::toLower(func.name) == "countdistinct") + func.name = count_distinct; + } +}; + +using CustomizeFunctionsMatcher = OneTypeMatcher; +using CustomizeFunctionsVisitor = InDepthNodeVisitor; + /// Add columns from storage to source_columns list. void collectSourceColumns(const ColumnsDescription & columns, NamesAndTypesList & source_columns, bool add_virtuals) @@ -91,7 +112,10 @@ void collectSourceColumns(const ColumnsDescription & columns, NamesAndTypesList } } -std::vector getTablesWithColumns(const ASTSelectQuery & select_query, const Context & context) +std::vector getTablesWithColumns(const ASTSelectQuery & select_query, const Context & context, + const ASTTablesInSelectQueryElement * table_join_node, + NamesAndTypesList & columns_from_joined_table, + std::function get_column_names) { std::vector tables_with_columns = getDatabaseAndTablesWithColumnNames(select_query, context); @@ -104,6 +128,27 @@ std::vector getTablesWithColumns(const ASTSelectQuery & se ErrorCodes::ALIAS_REQUIRED); } + TableWithColumnNames joined_table; + + if (table_join_node) + { + const auto & joined_expression = table_join_node->table_expression->as(); + + columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context); + + joined_table.first = DatabaseAndTableWithAlias(joined_expression, context.getCurrentDatabase()); + for (const auto & column : columns_from_joined_table) + joined_table.second.push_back(column.name); + } + + /// If empty make table(s) with list of source and joined columns + if (tables_with_columns.empty()) + { + tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, get_column_names()); + if (!joined_table.second.empty()) + tables_with_columns.emplace_back(std::move(joined_table)); + } + return tables_with_columns; } @@ -220,10 +265,10 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, } /// Replacing scalar subqueries with constant values. -void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t subquery_depth) +void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t subquery_depth, Scalars & scalars) { LogAST log; - ExecuteScalarSubqueriesVisitor::Data visitor_data{context, subquery_depth}; + ExecuteScalarSubqueriesVisitor::Data visitor_data{context, subquery_depth, scalars}; ExecuteScalarSubqueriesVisitor(visitor_data, log.stream()).visit(query); } @@ -266,11 +311,36 @@ const std::unordered_set possibly_injective_function_names "dictGetDateTime" }; +/** You can not completely remove GROUP BY. Because if there were no aggregate functions, then it turns out that there will be no aggregation. + * Instead, leave `GROUP BY const`. + * Next, see deleting the constants in the analyzeAggregation method. + */ +void appendUnusedGroupByColumn(ASTSelectQuery * select_query, const NameSet & source_columns) +{ + /// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens. + UInt64 unused_column = 0; + String unused_column_name = toString(unused_column); + + while (source_columns.count(unused_column_name)) + { + ++unused_column; + unused_column_name = toString(unused_column); + } + + select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::make_shared()); + select_query->groupBy()->children.emplace_back(std::make_shared(UInt64(unused_column))); +} + /// Eliminates injective function calls and constant expressions from group by statement. void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, const Context & context) { if (!select_query->groupBy()) + { + // If there is a HAVING clause without GROUP BY, make sure we have some aggregation happen. + if (select_query->having()) + appendUnusedGroupByColumn(select_query, source_columns); return; + } const auto is_literal = [] (const ASTPtr & ast) -> bool { @@ -345,25 +415,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum } if (group_exprs.empty()) - { - /** You can not completely remove GROUP BY. Because if there were no aggregate functions, then it turns out that there will be no aggregation. - * Instead, leave `GROUP BY const`. - * Next, see deleting the constants in the analyzeAggregation method. - */ - - /// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens. - UInt64 unused_column = 0; - String unused_column_name = toString(unused_column); - - while (source_columns.count(unused_column_name)) - { - ++unused_column; - unused_column_name = toString(unused_column); - } - - select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::make_shared()); - select_query->groupBy()->children.emplace_back(std::make_shared(UInt64(unused_column))); - } + appendUnusedGroupByColumn(select_query, source_columns); } /// Remove duplicate items from ORDER BY. @@ -541,11 +593,16 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s } } -void replaceJoinedTable(const ASTTablesInSelectQueryElement* join) +void replaceJoinedTable(const ASTTablesInSelectQueryElement * join) { if (!join || !join->table_expression) return; + /// TODO: Push down for CROSS JOIN is not OK [disabled] + const auto & table_join = join->table_join->as(); + if (table_join.kind == ASTTableJoin::Kind::Cross) + return; + auto & table_expr = join->table_expression->as(); if (table_expr.database_and_table_name) { @@ -805,41 +862,34 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( if (remove_duplicates) renameDuplicatedColumns(select_query); - if (const ASTTablesInSelectQueryElement * node = select_query->join()) + const ASTTablesInSelectQueryElement * table_join_node = select_query->join(); + if (table_join_node) { if (!settings.any_join_distinct_right_table_keys) - checkJoin(node); + checkJoin(table_join_node); if (settings.enable_optimize_predicate_expression) - replaceJoinedTable(node); - - const auto & joined_expression = node->table_expression->as(); - DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase()); - - result.analyzed_join->columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context); - result.analyzed_join->deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix()); + replaceJoinedTable(table_join_node); } - auto tables_with_columns = getTablesWithColumns(*select_query, context); - - /// If empty make fake table with list of source and joined columns - if (tables_with_columns.empty()) + auto get_column_names = [&]() -> Names { - Names columns_list; if (storage) - columns_list = storage->getColumns().getOrdinary().getNames(); - else - { - columns_list.reserve(result.source_columns.size()); - for (const auto & column : result.source_columns) - columns_list.emplace_back(column.name); - } + return storage->getColumns().getOrdinary().getNames(); - for (auto & column : result.analyzed_join->getQualifiedColumnsSet()) - columns_list.emplace_back(column); + Names columns; + columns.reserve(result.source_columns.size()); + for (const auto & column : result.source_columns) + columns.push_back(column.name); + return columns; + }; - tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(columns_list)); - } + auto tables_with_columns = getTablesWithColumns(*select_query, context, table_join_node, + result.analyzed_join->columns_from_joined_table, get_column_names); + + if (tables_with_columns.size() > 1) + result.analyzed_join->deduplicateAndQualifyColumnNames( + source_columns_set, tables_with_columns[1].first.getQualifiedNamePrefix()); translateQualifiedNames(query, *select_query, source_columns_set, std::move(tables_with_columns)); @@ -850,6 +900,11 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( LogicalExpressionsOptimizer(select_query, settings.optimize_min_equality_disjunction_chain_length.value).perform(); } + { + CustomizeFunctionsVisitor::Data data{settings.count_distinct_implementation}; + CustomizeFunctionsVisitor(data).visit(query); + } + /// Creates a dictionary `aliases`: alias -> ASTPtr { LogAST log; @@ -857,6 +912,12 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( QueryAliasesVisitor(query_aliases_data, log.stream()).visit(query); } + /// Mark table ASTIdentifiers with not a column marker + { + MarkTableIdentifiersVisitor::Data data{result.aliases}; + MarkTableIdentifiersVisitor(data).visit(query); + } + /// Common subexpression elimination. Rewrite rules. { QueryNormalizer::Data normalizer_data(result.aliases, context.getSettingsRef()); @@ -871,7 +932,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( removeUnneededColumnsFromSelectClause(select_query, required_result_columns, remove_duplicates); /// Executing scalar subqueries - replacing them with constant values. - executeScalarSubqueries(query, context, subquery_depth); + executeScalarSubqueries(query, context, subquery_depth, result.scalars); /// Optimize if with constant condition after constants was substituted instead of scalar subqueries. OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index 44fdc61ded3..96f5678ac6f 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -14,6 +15,7 @@ class ASTFunction; class AnalyzedJoin; class Context; struct SelectQueryOptions; +using Scalars = std::map; struct SyntaxAnalyzerResult { @@ -43,8 +45,12 @@ struct SyntaxAnalyzerResult /// Predicate optimizer overrides the sub queries bool rewrite_subqueries = false; + /// Results of scalar sub queries + Scalars scalars; + void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns); Names requiredSourceColumns() const { return required_source_columns.getNames(); } + const Scalars & getScalars() const { return scalars; } }; using SyntaxAnalyzerResultPtr = std::shared_ptr; diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 7ae98d3e9c8..df0946f098a 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -61,24 +61,20 @@ void TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr &, { if (IdentifierSemantic::getColumnName(identifier)) { - size_t best_table_pos = 0; - size_t best_match = 0; - for (size_t i = 0; i < data.tables.size(); ++i) - if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i].first)) - if (match > best_match) - { - best_match = match; - best_table_pos = i; - } + String short_name = identifier.shortName(); + size_t table_pos = 0; + bool allow_ambiguous = data.join_using_columns.count(short_name); + if (IdentifierSemantic::chooseTable(identifier, data.tables, table_pos, allow_ambiguous)) + { + IdentifierSemantic::setMembership(identifier, table_pos); - if (best_match) - IdentifierSemantic::setMembership(identifier, best_table_pos + 1); - - /// In case if column from the joined table are in source columns, change it's name to qualified. - if (best_table_pos && data.source_columns.count(identifier.shortName())) - IdentifierSemantic::setNeedLongName(identifier, true); - if (!data.tables.empty()) - IdentifierSemantic::setColumnNormalName(identifier, data.tables[best_table_pos].first); + /// In case if column from the joined table are in source columns, change it's name to qualified. + auto & table = data.tables[table_pos].first; + if (table_pos && data.hasColumn(short_name)) + IdentifierSemantic::setColumnLongName(identifier, table); + else + IdentifierSemantic::setColumnShortName(identifier, table); + } } } @@ -134,8 +130,10 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr Visitor(data).visit(select.refHaving()); } -static void addIdentifier(ASTs & nodes, const String & table_name, const String & column_name, AsteriskSemantic::RevertedAliasesPtr aliases) +static void addIdentifier(ASTs & nodes, const DatabaseAndTableWithAlias & table, const String & column_name, + AsteriskSemantic::RevertedAliasesPtr aliases) { + String table_name = table.getQualifiedNamePrefix(false); auto identifier = std::make_shared(std::vector{table_name, column_name}); bool added = false; @@ -197,8 +195,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { if (first_table || !data.join_using_columns.count(column_name)) { - String table_name = table.getQualifiedNamePrefix(false); - addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk)); + addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*asterisk)); } } @@ -214,8 +211,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { if (asterisk_pattern->isColumnMatching(column_name) && (first_table || !data.join_using_columns.count(column_name))) { - String table_name = table.getQualifiedNamePrefix(false); - addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*asterisk_pattern)); + addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*asterisk_pattern)); } } @@ -232,8 +228,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { for (const auto & column_name : table_columns) { - String table_name = table.getQualifiedNamePrefix(false); - addIdentifier(node.children, table_name, column_name, AsteriskSemantic::getAliases(*qualified_asterisk)); + addIdentifier(node.children, table, column_name, AsteriskSemantic::getAliases(*qualified_asterisk)); } break; } @@ -269,11 +264,13 @@ void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, D void RestoreQualifiedNamesData::visit(ASTIdentifier & identifier, ASTPtr & ast) { - if (IdentifierSemantic::getColumnName(identifier) && - IdentifierSemantic::getMembership(identifier)) + if (IdentifierSemantic::getColumnName(identifier)) { - ast = identifier.clone(); - ast->as()->restoreCompoundName(); + if (IdentifierSemantic::getMembership(identifier)) + { + ast = identifier.clone(); + ast->as()->restoreCompoundName(); + } } } diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h index 4bf18b59cb9..b3718170dda 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -35,6 +35,10 @@ public: , has_columns(has_columns_) {} + bool hasColumn(const String & name) const { return source_columns.count(name); } + bool hasTable() const { return !tables.empty(); } + bool processAsterisks() const { return hasTable() && has_columns; } + static std::vector tablesOnly(const std::vector & tables) { std::vector tables_with_columns; @@ -44,8 +48,6 @@ public: tables_with_columns.emplace_back(TableWithColumnNames{table, {}}); return tables_with_columns; } - - bool processAsterisks() const { return !tables.empty() && has_columns; } }; static void visit(ASTPtr & ast, Data & data); diff --git a/dbms/src/Interpreters/misc.h b/dbms/src/Interpreters/misc.h new file mode 100644 index 00000000000..d5e2894bb4c --- /dev/null +++ b/dbms/src/Interpreters/misc.h @@ -0,0 +1,16 @@ +#pragma once + +namespace DB +{ + +inline bool functionIsInOperator(const std::string & name) +{ + return name == "in" || name == "notIn"; +} + +inline bool functionIsInOrGlobalInOperator(const std::string & name) +{ + return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn"; +} + +} diff --git a/dbms/src/Interpreters/tests/create_query.cpp b/dbms/src/Interpreters/tests/create_query.cpp index 47e1f202db7..fc487f4b7bb 100644 --- a/dbms/src/Interpreters/tests/create_query.cpp +++ b/dbms/src/Interpreters/tests/create_query.cpp @@ -84,7 +84,7 @@ try context.setPath("./"); auto database = std::make_shared("test", "./metadata/test/", context); context.addDatabase("test", database); - database->loadTables(context, false); + database->loadStoredObjects(context, false); context.setCurrentDatabase("test"); InterpreterCreateQuery interpreter(ast, context); diff --git a/dbms/src/Interpreters/tests/select_query.cpp b/dbms/src/Interpreters/tests/select_query.cpp index 1283ae6e659..54613fffd8e 100644 --- a/dbms/src/Interpreters/tests/select_query.cpp +++ b/dbms/src/Interpreters/tests/select_query.cpp @@ -39,7 +39,7 @@ try DatabasePtr system = std::make_shared("system", "./metadata/system/", context); context.addDatabase("system", system); - system->loadTables(context, false); + system->loadStoredObjects(context, false); attachSystemTablesLocal(*context.getDatabase("system")); context.setCurrentDatabase("default"); diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index bc4a8290d8d..8c3e9163caa 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -238,10 +238,10 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat } else { - /// Always CREATE and always DICTIONARY - settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE DICTIONARY " << (if_not_exists ? "IF NOT EXISTS " : "") - << (settings.hilite ? hilite_none : "") << (!database.empty() ? backQuoteIfNeed(database) + "." : "") - << backQuoteIfNeed(table); + /// Always DICTIONARY + settings.ostr << (settings.hilite ? hilite_keyword : "") << (attach ? "ATTACH " : "CREATE ") << "DICTIONARY " + << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); } if (as_table_function) diff --git a/dbms/src/Parsers/ASTDictionary.cpp b/dbms/src/Parsers/ASTDictionary.cpp index 532190ae2d2..ec750acff31 100644 --- a/dbms/src/Parsers/ASTDictionary.cpp +++ b/dbms/src/Parsers/ASTDictionary.cpp @@ -24,8 +24,7 @@ void ASTDictionaryRange::formatImpl(const FormatSettings & settings, << "(" << (settings.hilite ? hilite_keyword : "") << "MIN " - << (settings.hilite ? hilite_none : "") - << min_attr_name << ", " + << min_attr_name << " " << (settings.hilite ? hilite_keyword : "") << "MAX " << (settings.hilite ? hilite_none : "") @@ -54,8 +53,7 @@ void ASTDictionaryLifetime::formatImpl(const FormatSettings & settings, << "(" << (settings.hilite ? hilite_keyword : "") << "MIN " - << (settings.hilite ? hilite_none : "") - << min_sec << ", " + << min_sec << " " << (settings.hilite ? hilite_keyword : "") << "MAX " << (settings.hilite ? hilite_none : "") @@ -133,14 +131,17 @@ void ASTDictionary::formatImpl(const FormatSettings & settings, FormatState & st if (primary_key) { settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "PRIMARY KEY " - << (settings.hilite ? hilite_none : ""); + << (settings.hilite ? hilite_none : ""); primary_key->formatImpl(settings, state, frame); } if (source) - settings.ostr << settings.nl_or_ws; - - source->formatImpl(settings, state, frame); + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "SOURCE(" + << (settings.hilite ? hilite_none : ""); + source->formatImpl(settings, state, frame); + settings.ostr << ")"; + } if (lifetime) { diff --git a/dbms/src/Parsers/ASTIdentifier.cpp b/dbms/src/Parsers/ASTIdentifier.cpp index e3948f99f5b..6307db675fa 100644 --- a/dbms/src/Parsers/ASTIdentifier.cpp +++ b/dbms/src/Parsers/ASTIdentifier.cpp @@ -34,10 +34,20 @@ ASTIdentifier::ASTIdentifier(const String & name_, std::vector && name_p , name_parts(name_parts_) , semantic(std::make_shared()) { + if (name_parts.size() && name_parts[0] == "") + name_parts.erase(name_parts.begin()); + + if (name == "") + { + if (name_parts.size() == 2) + name = name_parts[0] + '.' + name_parts[1]; + else if (name_parts.size() == 1) + name = name_parts[0]; + } } ASTIdentifier::ASTIdentifier(std::vector && name_parts_) - : ASTIdentifier(name_parts_.at(0) + '.' + name_parts_.at(1), std::move(name_parts_)) + : ASTIdentifier("", std::move(name_parts_)) {} void ASTIdentifier::setShortName(const String & new_name) diff --git a/dbms/src/Parsers/ASTShowTablesQuery.cpp b/dbms/src/Parsers/ASTShowTablesQuery.cpp index 34a8c9fb76a..7742cfcf06b 100644 --- a/dbms/src/Parsers/ASTShowTablesQuery.cpp +++ b/dbms/src/Parsers/ASTShowTablesQuery.cpp @@ -22,7 +22,8 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format } else { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") << "TABLES" << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") << + (dictionaries ? "DICTIONARIES" : "TABLES") << (settings.hilite ? hilite_none : ""); if (!from.empty()) settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") @@ -41,4 +42,3 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format } } - diff --git a/dbms/src/Parsers/ASTShowTablesQuery.h b/dbms/src/Parsers/ASTShowTablesQuery.h index f3500f437c3..fd0d5ff6379 100644 --- a/dbms/src/Parsers/ASTShowTablesQuery.h +++ b/dbms/src/Parsers/ASTShowTablesQuery.h @@ -15,6 +15,7 @@ class ASTShowTablesQuery : public ASTQueryWithOutput { public: bool databases{false}; + bool dictionaries{false}; bool temporary{false}; String from; String like; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index ce1490d18bd..094d29628f7 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -823,6 +823,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create("CREATE"); + ParserKeyword s_attach("ATTACH"); ParserKeyword s_dictionary("DICTIONARY"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserIdentifier name_p; @@ -840,8 +841,14 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E ASTPtr attributes; ASTPtr dictionary; + bool attach = false; if (!s_create.ignore(pos, expected)) - return false; + { + if (s_attach.ignore(pos, expected)) + attach = true; + else + return false; + } if (s_if_not_exists.ignore(pos, expected)) if_not_exists = true; @@ -859,21 +866,25 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E return false; } - if (!s_left_paren.ignore(pos, expected)) - return false; + if (!attach) + { + if (!s_left_paren.ignore(pos, expected)) + return false; - if (!attributes_p.parse(pos, attributes, expected)) - return false; + if (!attributes_p.parse(pos, attributes, expected)) + return false; - if (!s_right_paren.ignore(pos, expected)) - return false; + if (!s_right_paren.ignore(pos, expected)) + return false; - if (!dictionary_p.parse(pos, dictionary, expected)) - return false; + if (!dictionary_p.parse(pos, dictionary, expected)) + return false; + } auto query = std::make_shared(); node = query; query->is_dictionary = true; + query->attach = attach; if (database) query->database = typeid_cast(*database).name; diff --git a/dbms/src/Parsers/ParserShowTablesQuery.cpp b/dbms/src/Parsers/ParserShowTablesQuery.cpp index 3fe43c4557d..7d443da4f9f 100644 --- a/dbms/src/Parsers/ParserShowTablesQuery.cpp +++ b/dbms/src/Parsers/ParserShowTablesQuery.cpp @@ -20,6 +20,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_tables("TABLES"); ParserKeyword s_databases("DATABASES"); + ParserKeyword s_dictionaries("DICTIONARIES"); ParserKeyword s_from("FROM"); ParserKeyword s_not("NOT"); ParserKeyword s_like("LIKE"); @@ -45,33 +46,36 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (s_temporary.ignore(pos)) query->temporary = true; - if (s_tables.ignore(pos, expected)) + if (!s_tables.ignore(pos, expected)) { - if (s_from.ignore(pos, expected)) - { - if (!name_p.parse(pos, database, expected)) - return false; - } - - if (s_not.ignore(pos, expected)) - query->not_like = true; - - if (s_like.ignore(pos, expected)) - { - if (!like_p.parse(pos, like, expected)) - return false; - } - else if (query->not_like) + if (s_dictionaries.ignore(pos, expected)) + query->dictionaries = true; + else return false; - - if (s_limit.ignore(pos, expected)) - { - if (!limit_p.parse(pos, query->limit_length, expected)) - return false; - } } - else + + if (s_from.ignore(pos, expected)) + { + if (!name_p.parse(pos, database, expected)) + return false; + } + + if (s_not.ignore(pos, expected)) + query->not_like = true; + + if (s_like.ignore(pos, expected)) + { + if (!like_p.parse(pos, like, expected)) + return false; + } + else if (query->not_like) return false; + + if (s_limit.ignore(pos, expected)) + { + if (!limit_p.parse(pos, query->limit_length, expected)) + return false; + } } tryGetIdentifierNameInto(database, query->from); diff --git a/dbms/src/Parsers/tests/gtest_dictionary_parser.cpp b/dbms/src/Parsers/tests/gtest_dictionary_parser.cpp index 6d3c964bc45..934eb10f9a6 100644 --- a/dbms/src/Parsers/tests/gtest_dictionary_parser.cpp +++ b/dbms/src/Parsers/tests/gtest_dictionary_parser.cpp @@ -291,7 +291,7 @@ TEST(ParserDictionaryDDL, Formatting) ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); ASTCreateQuery * create = ast->as(); auto str = serializeAST(*create, true); - EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '') LIFETIME(MIN 1, MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column, MAX third_column)"); + EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)"); } TEST(ParserDictionaryDDL, ParseDropQuery) diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 5dce68ec381..ee3ebfd9964 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -81,12 +81,27 @@ void DistributedBlockOutputStream::writePrefix() void DistributedBlockOutputStream::write(const Block & block) { - if (insert_sync) - writeSync(block); - else - writeAsync(block); -} + Block ordinary_block{ block }; + /* They are added by the AddingDefaultBlockOutputStream, and we will get + * different number of columns eventually */ + for (const auto & col : storage.getColumns().getMaterialized()) + { + if (ordinary_block.has(col.name)) + { + ordinary_block.erase(col.name); + LOG_DEBUG(log, storage.getTableName() + << ": column " + col.name + " will be removed, " + << "because it is MATERIALIZED"); + } + } + + + if (insert_sync) + writeSync(ordinary_block); + else + writeAsync(ordinary_block); +} void DistributedBlockOutputStream::writeAsync(const Block & block) { diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index b3e4c776605..a2789fe3063 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 082bc038a36..c2475395101 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2723,10 +2723,20 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & if (!disk) throw Exception("Disk " + name + " does not exists on policy " + storage_policy->getName(), ErrorCodes::UNKNOWN_DISK); - for (const auto & part : parts) + parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr) + { + return part_ptr->disk->getName() == disk->getName(); + }), parts.end()); + + if (parts.empty()) { - if (part->disk->getName() == disk->getName()) - throw Exception("Part " + part->name + " already on disk " + name, ErrorCodes::UNKNOWN_DISK); + String no_parts_to_move_message; + if (moving_part) + no_parts_to_move_message = "Part '" + partition_id + "' is already on disk '" + disk->getName() + "'"; + else + no_parts_to_move_message = "All parts of partition '" + partition_id + "' are already on disk '" + disk->getName() + "'"; + + throw Exception(no_parts_to_move_message, ErrorCodes::UNKNOWN_DISK); } if (!movePartsToSpace(parts, std::static_pointer_cast(disk))) @@ -2758,10 +2768,28 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String if (!volume) throw Exception("Volume " + name + " does not exists on policy " + storage_policy->getName(), ErrorCodes::UNKNOWN_DISK); - for (const auto & part : parts) - for (const auto & disk : volume->disks) - if (part->disk->getName() == disk->getName()) - throw Exception("Part " + part->name + " already on volume '" + name + "'", ErrorCodes::UNKNOWN_DISK); + parts.erase(std::remove_if(parts.begin(), parts.end(), [&](auto part_ptr) + { + for (const auto & disk : volume->disks) + { + if (part_ptr->disk->getName() == disk->getName()) + { + return true; + } + } + return false; + }), parts.end()); + + if (parts.empty()) + { + String no_parts_to_move_message; + if (moving_part) + no_parts_to_move_message = "Part '" + partition_id + "' is already on volume '" + volume->getName() + "'"; + else + no_parts_to_move_message = "All parts of partition '" + partition_id + "' are already on volume '" + volume->getName() + "'"; + + throw Exception(no_parts_to_move_message, ErrorCodes::UNKNOWN_DISK); + } if (!movePartsToSpace(parts, std::static_pointer_cast(volume))) throw Exception("Cannot move parts because moves are manually disabled.", ErrorCodes::ABORTED); diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 856354959f9..147071fc493 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 264c91cd890..da3f1df8130 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index a772e0a204b..dcca5baf311 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 2c289dd714e..dbcfcc57d75 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -1,7 +1,6 @@ #include #include -#include #include @@ -323,11 +322,13 @@ BlockInputStreams StorageDistributed::read( Block header = InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock(); + const Scalars & scalars = context.hasQueryContext() ? context.getQueryContext().getScalars() : Scalars{}; + ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? ClusterProxy::SelectStreamFactory( - header, processed_stage, remote_table_function_ptr, context.getExternalTables()) + header, processed_stage, remote_table_function_ptr, scalars, context.getExternalTables()) : ClusterProxy::SelectStreamFactory( - header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables()); + header, processed_stage, QualifiedTableName{remote_database, remote_table}, scalars, context.getExternalTables()); if (settings.optimize_skip_unused_shards) { diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 56ab949f30c..deaea288e7b 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -143,7 +143,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & { auto stage_in_source_tables = QueryProcessingStage::FetchColumns; - DatabaseIteratorPtr iterator = getDatabaseIterator(context); + DatabaseTablesIteratorPtr iterator = getDatabaseIterator(context); size_t selected_table_size = 0; @@ -353,7 +353,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr & query, bool has_virtual_column, bool get_lock, const String & query_id) const { StorageListWithLocks selected_tables; - DatabaseIteratorPtr iterator = getDatabaseIterator(global_context); + DatabaseTablesIteratorPtr iterator = getDatabaseIterator(global_context); auto virtual_column = ColumnString::create(); @@ -387,12 +387,12 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const ASTPtr } -DatabaseIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const +DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const { checkStackSize(); auto database = context.getDatabase(source_database); auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); }; - return database->getIterator(global_context, table_name_match); + return database->getTablesIterator(global_context, table_name_match); } diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index dbf5d219957..debcb4da58e 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -71,7 +71,7 @@ private: template StoragePtr getFirstTable(F && predicate) const; - DatabaseIteratorPtr getDatabaseIterator(const Context & context) const; + DatabaseTablesIteratorPtr getDatabaseIterator(const Context & context) const; protected: StorageMerge( diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index c1d5c827f30..31c7b1c45c3 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -292,7 +292,7 @@ BlockInputStreams StorageSystemColumns::read( const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesWithDictionaryTablesIterator(context); iterator->isValid(); iterator->next()) { const String & table_name = iterator->name(); storages.emplace(std::piecewise_construct, diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index c6f7d4ac9ae..c31d514cf08 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -9,6 +9,9 @@ #include #include #include +#include +#include +#include #include #include @@ -19,34 +22,40 @@ namespace DB NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() { return { - { "name", std::make_shared() }, - { "status", std::make_shared(ExternalLoader::getStatusEnumAllPossibleValues()) }, - { "origin", std::make_shared() }, - { "type", std::make_shared() }, - { "key", std::make_shared() }, - { "attribute.names", std::make_shared(std::make_shared()) }, - { "attribute.types", std::make_shared(std::make_shared()) }, - { "bytes_allocated", std::make_shared() }, - { "query_count", std::make_shared() }, - { "hit_rate", std::make_shared() }, - { "element_count", std::make_shared() }, - { "load_factor", std::make_shared() }, - { "source", std::make_shared() }, - { "loading_start_time", std::make_shared() }, - { "loading_duration", std::make_shared() }, + {"database", std::make_shared()}, + {"name", std::make_shared()}, + {"status", std::make_shared(ExternalLoader::getStatusEnumAllPossibleValues())}, + {"origin", std::make_shared()}, + {"type", std::make_shared()}, + {"key", std::make_shared()}, + {"attribute.names", std::make_shared(std::make_shared())}, + {"attribute.types", std::make_shared(std::make_shared())}, + {"bytes_allocated", std::make_shared()}, + {"query_count", std::make_shared()}, + {"hit_rate", std::make_shared()}, + {"element_count", std::make_shared()}, + {"load_factor", std::make_shared()}, + {"source", std::make_shared()}, + {"loading_start_time", std::make_shared()}, + {"loading_duration", std::make_shared()}, //{ "creation_time", std::make_shared() }, - { "last_exception", std::make_shared() }, + {"last_exception", std::make_shared()}, }; } -void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const +void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & /*query_info*/) const { const auto & external_dictionaries = context.getExternalDictionariesLoader(); for (const auto & [dict_name, load_result] : external_dictionaries.getCurrentLoadResults()) { size_t i = 0; - res_columns[i++]->insert(dict_name); + res_columns[i++]->insert(load_result.repository_name); + if (!load_result.repository_name.empty()) + res_columns[i++]->insert(dict_name.substr(load_result.repository_name.length() + 1)); + else + res_columns[i++]->insert(dict_name); + res_columns[i++]->insert(static_cast(load_result.status)); res_columns[i++]->insert(load_result.origin); diff --git a/dbms/src/Storages/System/StorageSystemGraphite.cpp b/dbms/src/Storages/System/StorageSystemGraphite.cpp index b8f04103e91..36839e06196 100644 --- a/dbms/src/Storages/System/StorageSystemGraphite.cpp +++ b/dbms/src/Storages/System/StorageSystemGraphite.cpp @@ -36,7 +36,7 @@ StorageSystemGraphite::Configs StorageSystemGraphite::getConfigs(const Context & 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(); diff --git a/dbms/src/Storages/System/StorageSystemMutations.cpp b/dbms/src/Storages/System/StorageSystemMutations.cpp index 7bce6628c0e..0af32466d88 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.cpp +++ b/dbms/src/Storages/System/StorageSystemMutations.cpp @@ -43,7 +43,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex if (db.second->getEngineName() == "Lazy") continue; if (context.hasDatabaseAccessRights(db.first)) - for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) if (dynamic_cast(iterator->table().get())) merge_tree_tables[db.first][iterator->name()] = iterator->table(); } diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 0cf5f5e7013..2e4f5dee01f 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -104,7 +104,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { String table_name = iterator->name(); StoragePtr storage = iterator->table(); diff --git a/dbms/src/Storages/System/StorageSystemReplicas.cpp b/dbms/src/Storages/System/StorageSystemReplicas.cpp index 3e5c11c7787..3e319e19bd7 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicas.cpp @@ -71,7 +71,7 @@ BlockInputStreams StorageSystemReplicas::read( continue; if (context.hasDatabaseAccessRights(db.first)) { - for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) if (dynamic_cast(iterator->table().get())) replicated_tables[db.first][iterator->name()] = iterator->table(); } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp index e29900295d8..5d6aa412152 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -57,7 +57,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const if (context.hasDatabaseAccessRights(db.first)) { - for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) if (dynamic_cast(iterator->table().get())) replicated_tables[db.first][iterator->name()] = iterator->table(); } diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 27861fcb909..01f8704f681 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -86,7 +86,11 @@ public: UInt64 max_block_size_, ColumnPtr databases_, const Context & context_) - : columns_mask(std::move(columns_mask_)), header(std::move(header_)), max_block_size(max_block_size_), databases(std::move(databases_)), context(context_) {} + : columns_mask(std::move(columns_mask_)) + , header(std::move(header_)) + , max_block_size(max_block_size_) + , databases(std::move(databases_)) + , context(context_) {} String getName() const override { return "Tables"; } Block getHeader() const override { return header; } @@ -189,7 +193,7 @@ protected: } if (!tables_it || !tables_it->isValid()) - tables_it = database->getIterator(context); + tables_it = database->getTablesWithDictionaryTablesIterator(context); const bool need_lock_structure = needLockStructure(database, header); @@ -251,10 +255,10 @@ protected: } if (columns_mask[src_index++]) - res_columns[res_index++]->insert(database->getTableMetadataPath(table_name)); + res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name)); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(static_cast(database->getTableMetadataModificationTime(context, table_name))); + res_columns[res_index++]->insert(static_cast(database->getObjectMetadataModificationTime(context, table_name))); { Array dependencies_table_name_array; @@ -372,7 +376,7 @@ private: UInt64 max_block_size; ColumnPtr databases; size_t database_idx = 0; - DatabaseIteratorPtr tables_it; + DatabaseTablesIteratorPtr tables_it; const Context context; bool done = false; DatabasePtr database; diff --git a/dbms/src/TableFunctions/TableFunctionMerge.cpp b/dbms/src/TableFunctions/TableFunctionMerge.cpp index b5d15707e89..0cae1cda987 100644 --- a/dbms/src/TableFunctions/TableFunctionMerge.cpp +++ b/dbms/src/TableFunctions/TableFunctionMerge.cpp @@ -33,7 +33,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str { auto database = context.getDatabase(source_database); - auto iterator = database->getIterator(context, table_name_match); + auto iterator = database->getTablesIterator(context, table_name_match); if (iterator->isValid()) any_table = iterator->table(); diff --git a/dbms/tests/integration/test_dictionaries_ddl/__init__.py b/dbms/tests/integration/test_dictionaries_ddl/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_dictionaries_ddl/configs/config.xml b/dbms/tests/integration/test_dictionaries_ddl/configs/config.xml new file mode 100644 index 00000000000..e24857fa806 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_ddl/configs/config.xml @@ -0,0 +1,19 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + + 9000 + 127.0.0.1 + 500 + 5368709120 + ./clickhouse/ + users.xml + /etc/clickhouse-server/config.d/*.xml + diff --git a/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/dictionary_with_conflict_name.xml b/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/dictionary_with_conflict_name.xml new file mode 100644 index 00000000000..75e6f8953eb --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/dictionary_with_conflict_name.xml @@ -0,0 +1,41 @@ + + + test.conflicting_dictionary + + + localhost + 9000 + default + + test + xml_dictionary_table
+
+ + + + 0 + 0 + + + + 128 + + + + + id + + + SomeValue1 + UInt8 + 1 + + + + SomeValue2 + String + '' + + +
+
diff --git a/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load.xml b/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load.xml new file mode 100644 index 00000000000..d01f7a0155b --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/lazy_load.xml @@ -0,0 +1,4 @@ + + false + + diff --git a/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/simple_dictionary.xml b/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/simple_dictionary.xml new file mode 100644 index 00000000000..5032ca4266c --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_ddl/configs/dictionaries/simple_dictionary.xml @@ -0,0 +1,41 @@ + + + xml_dictionary + + + localhost + 9000 + default + + test + xml_dictionary_table
+
+ + + + 0 + 0 + + + + 128 + + + + + id + + + SomeValue1 + UInt8 + 1 + + + + SomeValue2 + String + '' + + +
+
diff --git a/dbms/tests/integration/test_dictionaries_ddl/configs/users.xml b/dbms/tests/integration/test_dictionaries_ddl/configs/users.xml new file mode 100644 index 00000000000..3e53e05aee1 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_ddl/configs/users.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + ::/0 + + default + default + + default + test + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_dictionaries_ddl/test.py b/dbms/tests/integration/test_dictionaries_ddl/test.py new file mode 100644 index 00000000000..a949bee136a --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_ddl/test.py @@ -0,0 +1,184 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV, assert_eq_with_retry +from helpers.client import QueryRuntimeException +import pymysql +import warnings +import time + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +node1 = cluster.add_instance('node1', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml']) +node2 = cluster.add_instance('node2', with_mysql=True, main_configs=['configs/dictionaries/simple_dictionary.xml', 'configs/dictionaries/lazy_load.xml']) +node3 = cluster.add_instance('node3', main_configs=['configs/dictionaries/dictionary_with_conflict_name.xml']) + + +def create_mysql_conn(user, password, hostname, port): + return pymysql.connect( + user=user, + password=password, + host=hostname, + port=port) + +def execute_mysql_query(connection, query): + with warnings.catch_warnings(): + warnings.simplefilter("ignore") + with connection.cursor() as cursor: + cursor.execute(query) + connection.commit() + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + for clickhouse in [node1, node2, node3]: + clickhouse.query("CREATE DATABASE test", user="admin") + clickhouse.query("CREATE TABLE test.xml_dictionary_table (id UInt64, SomeValue1 UInt8, SomeValue2 String) ENGINE = MergeTree() ORDER BY id", user="admin") + clickhouse.query("INSERT INTO test.xml_dictionary_table SELECT number, number % 23, hex(number) from numbers(1000)", user="admin") + yield cluster + + finally: + cluster.shutdown() + + + +@pytest.mark.parametrize("clickhouse,name,layout", [ + (node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), + (node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), + (node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), + (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), +]) +def test_crete_and_select_mysql(started_cluster, clickhouse, name, layout): + mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) + execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") + execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format(name)) + values = [] + for i in range(1000): + values.append('(' + ','.join([str(i), str(i * i), str(i) * 5, str(i * 3.14)]) + ')') + execute_mysql_query(mysql_conn, "INSERT INTO clickhouse.{} VALUES ".format(name) + ','.join(values)) + + clickhouse.query(""" + CREATE DICTIONARY default.{} ( + key_field1 Int32, + key_field2 Int64, + value1 String DEFAULT 'xxx', + value2 Float32 DEFAULT 'yyy' + ) + PRIMARY KEY key_field1, key_field2 + SOURCE(MYSQL( + USER 'root' + PASSWORD 'clickhouse' + DB 'clickhouse' + TABLE '{}' + REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333) + REPLICA(PRIORITY 2 HOST 'mysql1' PORT 3306) + )) + {} + LIFETIME(MIN 1 MAX 3) + """.format(name, name, layout)) + + for i in range(172, 200): + assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 5 + '\n' + stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip() + value = float(stroka) + assert int(value) == int(i * 3.14) + + + for i in range(1000): + values.append('(' + ','.join([str(i), str(i * i), str(i) * 3, str(i * 2.718)]) + ')') + execute_mysql_query(mysql_conn, "REPLACE INTO clickhouse.{} VALUES ".format(name) + ','.join(values)) + + clickhouse.query("SYSTEM RELOAD DICTIONARY 'default.{}'".format(name)) + + for i in range(172, 200): + assert clickhouse.query("SELECT dictGetString('default.{}', 'value1', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)) == str(i) * 3 + '\n' + stroka = clickhouse.query("SELECT dictGetFloat32('default.{}', 'value2', tuple(toInt32({}), toInt64({})))".format(name, i, i * i)).strip() + value = float(stroka) + assert int(value) == int(i * 2.718) + + clickhouse.query("select dictGetUInt8('xml_dictionary', 'SomeValue1', toUInt64(17))") == "17\n" + clickhouse.query("select dictGetString('xml_dictionary', 'SomeValue2', toUInt64(977))") == str(hex(977))[2:] + '\n' + + +def test_restricted_database(started_cluster): + for node in [node1, node2]: + node.query("CREATE DATABASE IF NOT EXISTS restricted_db", user="admin") + node.query("CREATE TABLE restricted_db.table_in_restricted_db AS test.xml_dictionary_table", user="admin") + + with pytest.raises(QueryRuntimeException): + node1.query(""" + CREATE DICTIONARY restricted_db.some_dict( + id UInt64, + SomeValue1 UInt8, + SomeValue2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db')) + LIFETIME(MIN 1 MAX 10) + """) + + with pytest.raises(QueryRuntimeException): + node1.query(""" + CREATE DICTIONARY default.some_dict( + id UInt64, + SomeValue1 UInt8, + SomeValue2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db')) + LIFETIME(MIN 1 MAX 10) + """) + + node1.query("SELECT dictGetUInt8('default.some_dict', 'SomeValue1', toUInt64(17))") == "17\n" + + # with lazy load we don't need query to get exception + with pytest.raises(QueryRuntimeException): + node2.query(""" + CREATE DICTIONARY restricted_db.some_dict( + id UInt64, + SomeValue1 UInt8, + SomeValue2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db')) + LIFETIME(MIN 1 MAX 10) + """) + + with pytest.raises(QueryRuntimeException): + node2.query(""" + CREATE DICTIONARY default.some_dict( + id UInt64, + SomeValue1 UInt8, + SomeValue2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_in_restricted_db' DB 'restricted_db')) + LIFETIME(MIN 1 MAX 10) + """) + + +def test_conflicting_name(started_cluster): + assert node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n' + + with pytest.raises(QueryRuntimeException): + node3.query(""" + CREATE DICTIONARY test.conflicting_dictionary( + id UInt64, + SomeValue1 UInt8, + SomeValue2 String + ) + PRIMARY KEY id + LAYOUT(FLAT()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'xml_dictionary_table' DB 'test')) + LIFETIME(MIN 1 MAX 10) + """) + + # old version still works + node3.query("select dictGetUInt8('test.conflicting_dictionary', 'SomeValue1', toUInt64(17))") == '17\n' diff --git a/dbms/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml b/dbms/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml index d41ba6066c4..2e6a1f80a6d 100644 --- a/dbms/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml +++ b/dbms/tests/integration/test_multiple_disks/configs/config.d/storage_configuration.xml @@ -74,6 +74,27 @@ + + + + + default + 0 + + + external + + + jbod1 + 1024 + + + jbod2 + 1024000000 + + + + diff --git a/dbms/tests/integration/test_multiple_disks/test.py b/dbms/tests/integration/test_multiple_disks/test.py index 4ee337229c9..446eca88142 100644 --- a/dbms/tests/integration/test_multiple_disks/test.py +++ b/dbms/tests/integration/test_multiple_disks/test.py @@ -1,8 +1,9 @@ -import time +import json import pytest import random +import re import string -import json +import time from multiprocessing.dummy import Pool from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -128,6 +129,38 @@ def test_system_tables(start_cluster): "max_data_part_size": "20971520", "move_factor": 0.1, }, + { + "policy_name": "special_warning_policy", + "volume_name": "special_warning_zero_volume", + "volume_priority": "1", + "disks": ["default"], + "max_data_part_size": "0", + "move_factor": 0.1, + }, + { + "policy_name": "special_warning_policy", + "volume_name": "special_warning_default_volume", + "volume_priority": "2", + "disks": ["external"], + "max_data_part_size": "0", + "move_factor": 0.1, + }, + { + "policy_name": "special_warning_policy", + "volume_name": "special_warning_small_volume", + "volume_priority": "3", + "disks": ["jbod1"], + "max_data_part_size": "1024", + "move_factor": 0.1, + }, + { + "policy_name": "special_warning_policy", + "volume_name": "special_warning_big_volume", + "volume_priority": "4", + "disks": ["jbod2"], + "max_data_part_size": "1024000000", + "move_factor": 0.1, + }, ] clickhouse_policies_data = json.loads(node1.query("SELECT * FROM system.storage_policies WHERE policy_name != 'default' FORMAT JSON"))["data"] @@ -193,6 +226,28 @@ def get_random_string(length): def get_used_disks_for_table(node, table_name): return node.query("select disk_name from system.parts where table == '{}' and active=1 order by modification_time".format(table_name)).strip().split('\n') +def test_no_warning_about_zero_max_data_part_size(start_cluster): + def get_log(node): + return node.exec_in_container(["bash", "-c", "cat /var/log/clickhouse-server/clickhouse-server.log"]) + + for node in (node1, node2): + node.query(""" + CREATE TABLE default.test_warning_table ( + s String + ) ENGINE = MergeTree + ORDER BY tuple() + SETTINGS storage_policy='small_jbod_with_external' + """) + node.query(""" + DROP TABLE default.test_warning_table + """) + log = get_log(node) + assert not re.search("Warning.*Volume.*special_warning_zero_volume", log) + assert not re.search("Warning.*Volume.*special_warning_default_volume", log) + assert re.search("Warning.*Volume.*special_warning_small_volume", log) + assert not re.search("Warning.*Volume.*special_warning_big_volume", log) + + @pytest.mark.parametrize("name,engine", [ ("mt_on_jbod","MergeTree()"), ("replicated_mt_on_jbod","ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')",), @@ -462,7 +517,7 @@ def test_alter_move(start_cluster, name, engine): node1.query("INSERT INTO {} VALUES(toDate('2019-04-10'), 42)".format(name)) node1.query("INSERT INTO {} VALUES(toDate('2019-04-11'), 43)".format(name)) used_disks = get_used_disks_for_table(node1, name) - assert all(d.startswith("jbod") for d in used_disks), "All writes shoud go to jbods" + assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods" first_part = node1.query("SELECT name FROM system.parts WHERE table = '{}' and active = 1 ORDER BY modification_time LIMIT 1".format(name)).strip() @@ -498,6 +553,91 @@ def test_alter_move(start_cluster, name, engine): finally: node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + +@pytest.mark.parametrize("volume_or_disk", [ + "DISK", + "VOLUME" +]) +def test_alter_move_half_of_partition(start_cluster, volume_or_disk): + name = "alter_move_half_of_partition" + engine = "MergeTree()" + try: + node1.query(""" + CREATE TABLE {name} ( + EventDate Date, + number UInt64 + ) ENGINE = {engine} + ORDER BY tuple() + PARTITION BY toYYYYMM(EventDate) + SETTINGS storage_policy='jbods_with_external' + """.format(name=name, engine=engine)) + + node1.query("SYSTEM STOP MERGES {}".format(name)) + + node1.query("INSERT INTO {} VALUES(toDate('2019-03-15'), 65)".format(name)) + node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 42)".format(name)) + used_disks = get_used_disks_for_table(node1, name) + assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods" + + time.sleep(1) + parts = node1.query("SELECT name FROM system.parts WHERE table = '{}' and active = 1".format(name)).splitlines() + assert len(parts) == 2 + + node1.query("ALTER TABLE {} MOVE PART '{}' TO VOLUME 'external'".format(name, parts[0])) + disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and name = '{}' and active = 1".format(name, parts[0])).splitlines() + assert disks == ["external"] + + time.sleep(1) + node1.query("ALTER TABLE {} MOVE PARTITION 201903 TO {volume_or_disk} 'external'".format(name, volume_or_disk=volume_or_disk)) + disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201903' and active = 1".format(name)).splitlines() + assert disks == ["external"]*2 + + assert node1.query("SELECT COUNT() FROM {}".format(name)) == "2\n" + + finally: + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + + +@pytest.mark.parametrize("volume_or_disk", [ + "DISK", + "VOLUME" +]) +def test_alter_double_move_partition(start_cluster, volume_or_disk): + name = "alter_double_move_partition" + engine = "MergeTree()" + try: + node1.query(""" + CREATE TABLE {name} ( + EventDate Date, + number UInt64 + ) ENGINE = {engine} + ORDER BY tuple() + PARTITION BY toYYYYMM(EventDate) + SETTINGS storage_policy='jbods_with_external' + """.format(name=name, engine=engine)) + + node1.query("SYSTEM STOP MERGES {}".format(name)) + + node1.query("INSERT INTO {} VALUES(toDate('2019-03-15'), 65)".format(name)) + node1.query("INSERT INTO {} VALUES(toDate('2019-03-16'), 42)".format(name)) + used_disks = get_used_disks_for_table(node1, name) + assert all(d.startswith("jbod") for d in used_disks), "All writes should go to jbods" + + time.sleep(1) + node1.query("ALTER TABLE {} MOVE PARTITION 201903 TO {volume_or_disk} 'external'".format(name, volume_or_disk=volume_or_disk)) + disks = node1.query("SELECT disk_name FROM system.parts WHERE table = '{}' and partition = '201903' and active = 1".format(name)).splitlines() + assert disks == ["external"]*2 + + assert node1.query("SELECT COUNT() FROM {}".format(name)) == "2\n" + + time.sleep(1) + with pytest.raises(QueryRuntimeException): + node1.query("ALTER TABLE {} MOVE PARTITION 201903 TO {volume_or_disk} 'external'".format(name, volume_or_disk=volume_or_disk)) + + finally: + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + + def produce_alter_move(node, name): move_type = random.choice(["PART", "PARTITION"]) if move_type == "PART": diff --git a/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.reference b/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.reference index 24649ea3acb..df21becc999 100644 --- a/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.reference @@ -56,26 +56,26 @@ comma nullable 1 1 1 1 2 2 1 2 cross -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.a +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a cross nullable -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.a -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\n, t2_00826\nWHERE a = t2_00826.a +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a cross nullable vs not nullable -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = t2_00826.b -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = t2_00826.b +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b cross self SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nCROSS JOIN t1_00826 AS y\nWHERE (a = y.a) AND (b = y.b) SELECT \n a, \n b, \n y.a, \n y.b\nFROM t1_00826 AS x\nALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b)\nWHERE (a = y.a) AND (b = y.b) cross one table expr -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE a = b +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b cross multiple ands -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) cross and inside and -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n) AS t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) cross split conjunction -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0) -SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00826\n WHERE b > 0\n) AS t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0) +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0) +SELECT \n a, \n b, \n t2_00826.a, \n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0) diff --git a/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.sql b/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.sql index fa16cf398da..e21d257d2da 100644 --- a/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.sql +++ b/dbms/tests/queries/0_stateless/00826_cross_to_inner_join.sql @@ -1,9 +1,10 @@ SET enable_debug_queries = 1; +SET enable_optimize_predicate_expression = 0; set allow_experimental_cross_to_join_conversion = 0; -select * from system.one cross join system.one; +select * from system.one l cross join system.one r; set allow_experimental_cross_to_join_conversion = 1; -select * from system.one cross join system.one; +select * from system.one l cross join system.one r; DROP TABLE IF EXISTS t1_00826; DROP TABLE IF EXISTS t2_00826; diff --git a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference index e1256053739..453458a6ecf 100644 --- a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference +++ b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference @@ -1,17 +1,17 @@ -SELECT a\nFROM t1_00849\nCROSS JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 -SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a -SELECT a\nFROM t1_00849\nALL INNER JOIN \n(\n SELECT *\n FROM t2_00849\n) AS t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n WHERE (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n WHERE `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n WHERE (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = `--t2_00849.b`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n WHERE `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n WHERE (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = `--t1_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n WHERE `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n WHERE (`--t2_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849\n )\n CROSS JOIN \n (\n SELECT *\n FROM t3_00849\n ) AS t3_00849\n)\nCROSS JOIN \n(\n SELECT *\n FROM t4_00849\n) AS t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN \n (\n SELECT *\n FROM t2_00849\n ) AS t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN \n(\n SELECT *\n FROM t3_00849\n) AS t3_00849 +SELECT a\nFROM t1_00849\nCROSS JOIN t2_00849 +SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a +SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849 +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849 +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN t3_00849 SELECT * FROM t1, t2 1 1 1 1 1 1 1 \N diff --git a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql index d1d247a0174..f80daecbe87 100644 --- a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql +++ b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql @@ -1,4 +1,5 @@ SET enable_debug_queries = 1; +SET enable_optimize_predicate_expression = 0; SET joined_subquery_requires_alias = 0; DROP TABLE IF EXISTS t1_00849; diff --git a/dbms/tests/queries/0_stateless/00936_crc32_function.reference b/dbms/tests/queries/0_stateless/00936_crc_functions.reference similarity index 87% rename from dbms/tests/queries/0_stateless/00936_crc32_function.reference rename to dbms/tests/queries/0_stateless/00936_crc_functions.reference index 90c6a41551b..1431a2e654b 100644 --- a/dbms/tests/queries/0_stateless/00936_crc32_function.reference +++ b/dbms/tests/queries/0_stateless/00936_crc_functions.reference @@ -20,3 +20,7 @@ qwerty string 55151997 2663297705 qqq aaa 3142898280 4027020077 zxcqwer 3358319860 0 aasq xxz 3369829874 4069886758 +CRC32IEEE() +7332BC33 +CRC64() +72D5B9EA0B70CE1E diff --git a/dbms/tests/queries/0_stateless/00936_crc32_function.sql b/dbms/tests/queries/0_stateless/00936_crc_functions.sql similarity index 88% rename from dbms/tests/queries/0_stateless/00936_crc32_function.sql rename to dbms/tests/queries/0_stateless/00936_crc_functions.sql index 1bc9d9ec246..fd324ea23fa 100644 --- a/dbms/tests/queries/0_stateless/00936_crc32_function.sql +++ b/dbms/tests/queries/0_stateless/00936_crc_functions.sql @@ -18,3 +18,8 @@ select CRC32(str1), CRC32(str2) from table1 order by CRC32(str1), CRC32(str2); select str1, str2, CRC32(str1), CRC32(str2) from table1 order by CRC32(str1), CRC32(str2); DROP TABLE table1; + +SELECT 'CRC32IEEE()'; +SELECT hex(CRC32IEEE('foo')); +SELECT 'CRC64()'; +SELECT hex(CRC64('foo')); diff --git a/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference b/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference index b01acf34583..11b42f40c7a 100644 --- a/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference +++ b/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.reference @@ -1,3 +1,14 @@ +insert_distributed_sync=0 2018-08-01 2018-08-01 2018-08-01 2017-08-01 +2018-08-01 2017-08-01 +2018-08-01 +2018-08-01 2017-08-01 +insert_distributed_sync=1 +2018-08-01 +2018-08-01 +2018-08-01 2017-08-01 +2018-08-01 2017-08-01 +2018-08-01 +2018-08-01 2017-08-01 diff --git a/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql b/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql index 9e5bc3cbdf9..6b70d927204 100644 --- a/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql +++ b/dbms/tests/queries/0_stateless/00952_insert_into_distributed_with_materialized_column.sql @@ -1,15 +1,42 @@ DROP TABLE IF EXISTS local_00952; DROP TABLE IF EXISTS distributed_00952; -CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192); -CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_shard_localhost', currentDatabase(), local_00952, rand()); +-- +-- insert_distributed_sync=0 +-- +SELECT 'insert_distributed_sync=0'; +SET insert_distributed_sync=0; -SET insert_distributed_sync=1; +CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192); +CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_00952, rand()); INSERT INTO distributed_00952 VALUES ('2018-08-01'); +SYSTEM FLUSH DISTRIBUTED distributed_00952; + SELECT * FROM distributed_00952; +SELECT date, value FROM distributed_00952; SELECT * FROM local_00952; SELECT date, value FROM local_00952; DROP TABLE distributed_00952; DROP TABLE local_00952; + +-- +-- insert_distributed_sync=1 +-- +SELECT 'insert_distributed_sync=1'; +SET insert_distributed_sync=1; + +CREATE TABLE local_00952 (date Date, value Date MATERIALIZED toDate('2017-08-01')) ENGINE = MergeTree(date, date, 8192); +CREATE TABLE distributed_00952 AS local_00952 ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_00952, rand()); + +INSERT INTO distributed_00952 VALUES ('2018-08-01'); + +SELECT * FROM distributed_00952; +SELECT date, value FROM distributed_00952; +SELECT * FROM local_00952; +SELECT date, value FROM local_00952; + +DROP TABLE distributed_00952; +DROP TABLE local_00952; + diff --git a/dbms/tests/queries/0_stateless/01018_ambiguous_column.reference b/dbms/tests/queries/0_stateless/01018_ambiguous_column.reference new file mode 100644 index 00000000000..a2a1d6ea4f6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ambiguous_column.reference @@ -0,0 +1,13 @@ +0 0 +0 0 +0 +0 +0 +0 +┌─one.dummy─┬─A.dummy─┬─B.dummy─┐ +│ 0 │ 0 │ 0 │ +└───────────┴─────────┴─────────┘ +┌─A.dummy─┬─one.dummy─┬─two.dummy─┐ +│ 0 │ 0 │ 0 │ +└─────────┴───────────┴───────────┘ +0 diff --git a/dbms/tests/queries/0_stateless/01018_ambiguous_column.sql b/dbms/tests/queries/0_stateless/01018_ambiguous_column.sql new file mode 100644 index 00000000000..54603aab810 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ambiguous_column.sql @@ -0,0 +1,27 @@ +select * from system.one cross join system.one; -- { serverError 352 } +select * from system.one cross join system.one r; +select * from system.one l cross join system.one; +select * from system.one left join system.one using dummy; +select dummy from system.one left join system.one using dummy; + +USE system; + +SELECT dummy FROM one AS A JOIN one ON A.dummy = one.dummy; +SELECT dummy FROM one JOIN one AS A ON A.dummy = one.dummy; +SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; -- { serverError 352 } +SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError 352 } +SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; -- { serverError 352 } +SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError 352 } + +SELECT * from one +JOIN one A ON one.dummy = A.dummy +JOIN one B ON one.dummy = B.dummy +FORMAT PrettyCompact; + +SELECT * from one A +JOIN system.one one ON A.dummy = one.dummy +JOIN system.one two ON A.dummy = two.dummy +FORMAT PrettyCompact; + +-- SELECT one.dummy FROM one AS A FULL JOIN (SELECT 0 AS dymmy) AS one USING dummy; +SELECT one.dummy FROM one AS A JOIN (SELECT 0 AS dummy) B USING dummy; diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference new file mode 100644 index 00000000000..abc3218ce6c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.reference @@ -0,0 +1,7 @@ +2 +2 +2 +2 +2 +2 +2 diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh new file mode 100755 index 00000000000..782e6955b6e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_bad_queries.sh @@ -0,0 +1,120 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1" + +# Simple layout, but with two keys +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + PRIMARY KEY key1, key2 + LAYOUT(HASHED()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) + LIFETIME(MIN 1 MAX 10) +" 2>&1 | grep -c 'Primary key for simple dictionary must contain exactly one element' + + +# Simple layout, but with non existing key +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + PRIMARY KEY non_existing_column + LAYOUT(HASHED()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) + LIFETIME(MIN 1 MAX 10) +" 2>&1 | grep -c "Unknown key attribute 'non_existing_column'" + +# Complex layout, with non existing key +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + PRIMARY KEY non_existing_column, key1 + LAYOUT(COMPLEX_KEY_HASHED()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) + LIFETIME(MIN 1 MAX 10) +" 2>&1 | grep -c "Unknown key attribute 'non_existing_column'" + +# No layout +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + PRIMARY KEY key2, key1 + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) + LIFETIME(MIN 1 MAX 10) +" 2>&1 | grep -c "Cannot create dictionary with empty layout" + +# No PK +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + LAYOUT(COMPLEX_KEY_HASHED()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) + LIFETIME(MIN 1 MAX 10) +" 2>&1 | grep -c "Cannot create dictionary without primary key" + +# No lifetime +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + PRIMARY KEY key2, key1 + LAYOUT(COMPLEX_KEY_HASHED()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) +" 2>&1 | grep -c "Cannot create dictionary with empty lifetime" + +# No source +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + PRIMARY KEY non_existing_column, key1 + LAYOUT(COMPLEX_KEY_HASHED()) + LIFETIME(MIN 1 MAX 10) +" 2>&1 | grep -c "Cannot create dictionary with empty source" + + +# Complex layout, but with one key +$CLICKHOUSE_CLIENT -q " + CREATE DICTIONARY dict1 + ( + key1 UInt64, + key2 UInt64, + value String + ) + PRIMARY KEY key1 + LAYOUT(COMPLEX_KEY_HASHED()) + SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' DB '$CLICKHOUSE_DATABASE')) + LIFETIME(MIN 1 MAX 10) +" || exit 1 + + +$CLICKHOUSE_CLIENT -q "DROP DICTIONARY IF EXISTS dict1" diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.reference b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.reference new file mode 100644 index 00000000000..7193c3d3f3d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.reference @@ -0,0 +1 @@ +Still alive diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh new file mode 100755 index 00000000000..cc7d52ce1ab --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh @@ -0,0 +1,122 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -n -q " + DROP DATABASE IF EXISTS database_for_dict; + DROP TABLE IF EXISTS table_for_dict1; + DROP TABLE IF EXISTS table_for_dict2; + + CREATE TABLE table_for_dict1 (key_column UInt64, value_column String) ENGINE = MergeTree ORDER BY key_column; + CREATE TABLE table_for_dict2 (key_column UInt64, value_column String) ENGINE = MergeTree ORDER BY key_column; + + INSERT INTO table_for_dict1 SELECT number, toString(number) from numbers(1000); + INSERT INTO table_for_dict2 SELECT number, toString(number) from numbers(1000, 1000); + + CREATE DATABASE database_for_dict; + + CREATE DICTIONARY database_for_dict.dict1 (key_column UInt64, value_column String) PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) LIFETIME(MIN 1 MAX 5) LAYOUT(FLAT()); + + CREATE DICTIONARY database_for_dict.dict2 (key_column UInt64, value_column String) PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) LIFETIME(MIN 1 MAX 5) LAYOUT(CACHE(SIZE_IN_CELLS 150)); +" + + +function thread1() +{ + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.dictionaries FORMAT Null"; done +} + +function thread2() +{ + while true; do CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict1" ||: ; done +} + +function thread3() +{ + while true; do CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict2" ||:; done +} + + +function thread4() +{ + while true; do $CLICKHOUSE_CLIENT -n -q " + SELECT * FROM database_for_dict.dict1 FORMAT Null; + SELECT * FROM database_for_dict.dict2 FORMAT Null; + " ||: ; done +} + +function thread5() +{ + while true; do $CLICKHOUSE_CLIENT -n -q " + SELECT dictGetString('database_for_dict.dict1', 'value_column', toUInt64(number)) from numbers(1000) FROM FORMAT Null; + SELECT dictGetString('database_for_dict.dict2', 'value_column', toUInt64(number)) from numbers(1000) FROM FORMAT Null; + " ||: ; done +} + +function thread6() +{ + while true; do $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict1"; done +} + +function thread7() +{ + while true; do $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict2"; done +} + + +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; +export -f thread6; +export -f thread7; + +TIMEOUT=10 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & + +wait +$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'" + +$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY database_for_dict.dict1" +$CLICKHOUSE_CLIENT -q "ATTACH DICTIONARY database_for_dict.dict2" + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE table_for_dict1; + DROP TABLE table_for_dict2; + DROP DATABASE database_for_dict; +" diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference new file mode 100644 index 00000000000..327c02a4b8a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference @@ -0,0 +1,19 @@ +=DICTIONARY in Ordinary DB +CREATE DICTIONARY ordinary_db.dict1 (`key_column` UInt64 DEFAULT 0, `second_column` UInt8 DEFAULT 1, `third_column` String DEFAULT \'qqq\') PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict\')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()) +dict1 +1 +ordinary_db dict1 +==DETACH DICTIONARY +0 +==ATTACH DICTIONARY +dict1 +1 +ordinary_db dict1 +==DROP DICTIONARY +0 +=DICTIONARY in Memory DB +0 +=DICTIONARY in Lazy DB +=DROP DATABASE WITH DICTIONARY +dict4 +dict4 diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql new file mode 100644 index 00000000000..55d280cf045 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql @@ -0,0 +1,165 @@ +SET send_logs_level = 'none'; + +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict Engine = Ordinary; + +DROP TABLE IF EXISTS database_for_dict.table_for_dict; + +CREATE TABLE database_for_dict.table_for_dict +( + key_column UInt64, + second_column UInt8, + third_column String +) +ENGINE = MergeTree() +ORDER BY key_column; + +INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world'); + +DROP DATABASE IF EXISTS ordinary_db; + +CREATE DATABASE ordinary_db ENGINE = Ordinary; + +SELECT '=DICTIONARY in Ordinary DB'; + +DROP DICTIONARY IF EXISTS ordinary_db.dict1; + +CREATE DICTIONARY ordinary_db.dict1 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq' +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +SHOW CREATE DICTIONARY ordinary_db.dict1; + +SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; + +EXISTS DICTIONARY ordinary_db.dict1; + +SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; + +SELECT '==DETACH DICTIONARY'; +DETACH DICTIONARY ordinary_db.dict1; + +SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; + +EXISTS DICTIONARY ordinary_db.dict1; + +SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; + +SELECT '==ATTACH DICTIONARY'; +ATTACH DICTIONARY ordinary_db.dict1; + +SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; + +EXISTS DICTIONARY ordinary_db.dict1; + +SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; + +SELECT '==DROP DICTIONARY'; + +DROP DICTIONARY IF EXISTS ordinary_db.dict1; + +SYSTEM RELOAD DICTIONARY 'ordinary_db.dict1'; -- due to lazy_load at can persist for some time + +SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; + +EXISTS DICTIONARY ordinary_db.dict1; + +SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; + +DROP DATABASE IF EXISTS ordinary_db; + +DROP DATABASE IF EXISTS memory_db; + +CREATE DATABASE memory_db ENGINE = Memory; + +SELECT '=DICTIONARY in Memory DB'; + +CREATE DICTIONARY memory_db.dict2 +( + key_column UInt64 DEFAULT 0 INJECTIVE, + second_column UInt8 DEFAULT 1 EXPRESSION rand() % 222, + third_column String DEFAULT 'qqq' +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); -- {serverError 1} + +SHOW CREATE DICTIONARY memory_db.dict2; -- {serverError 487} + +SHOW DICTIONARIES FROM memory_db LIKE 'dict2'; + +EXISTS DICTIONARY memory_db.dict2; + +SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict2'; + +SELECT '=DICTIONARY in Lazy DB'; + +DROP DATABASE IF EXISTS lazy_db; + +CREATE DATABASE lazy_db ENGINE = Lazy(1); + +CREATE DICTIONARY lazy_db.dict3 +( + key_column UInt64 DEFAULT 0 INJECTIVE, + second_column UInt8 DEFAULT 1 EXPRESSION rand() % 222, + third_column String DEFAULT 'qqq' +) +PRIMARY KEY key_column, second_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(COMPLEX_KEY_HASHED()); -- {serverError 1} + +DROP DATABASE IF EXISTS lazy_db; + +SELECT '=DROP DATABASE WITH DICTIONARY'; + +DROP DATABASE IF EXISTS ordinary_db; + +CREATE DATABASE ordinary_db ENGINE = Ordinary; + +CREATE DICTIONARY ordinary_db.dict4 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq' +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +SHOW DICTIONARIES FROM ordinary_db; + +DROP DATABASE IF EXISTS ordinary_db; + +CREATE DATABASE ordinary_db ENGINE = Ordinary; + +SHOW DICTIONARIES FROM ordinary_db; + +CREATE DICTIONARY ordinary_db.dict4 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq' +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +SHOW DICTIONARIES FROM ordinary_db; + +DROP DATABASE IF EXISTS ordinary_db; + +DROP TABLE IF EXISTS database_for_dict.table_for_dict; + +DROP DATABASE IF EXISTS database_for_dict; diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference new file mode 100644 index 00000000000..a7f56f9ee03 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference @@ -0,0 +1,19 @@ +11 +11 +144 +144 +7 +7 +17 +11 +11 +7 +11 +6 +dict1 Dictionary +dict2 Dictionary +table_for_dict MergeTree +database_for_dict dict1 ComplexKeyCache +database_for_dict dict2 Hashed +6 +6 diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql new file mode 100644 index 00000000000..cb63ce86ada --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql @@ -0,0 +1,118 @@ +SET send_logs_level = 'none'; + +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict Engine = Ordinary; + +CREATE TABLE database_for_dict.table_for_dict +( + key_column UInt64, + second_column UInt8, + third_column String, + fourth_column Float64 +) +ENGINE = MergeTree() +ORDER BY key_column; + +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); + +DROP DICTIONARY IF EXISTS database_for_dict.dict1; + +CREATE DICTIONARY database_for_dict.dict1 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq', + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); +SELECT second_column FROM database_for_dict.dict1 WHERE key_column = 11; +SELECT dictGetString('database_for_dict.dict1', 'third_column', toUInt64(12)); +SELECT third_column FROM database_for_dict.dict1 WHERE key_column = 12; +SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', toUInt64(14)); +SELECT fourth_column FROM database_for_dict.dict1 WHERE key_column = 14; + +SELECT count(distinct(dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(number)))) from numbers(100); + +DETACH DICTIONARY database_for_dict.dict1; + +SYSTEM RELOAD DICTIONARY 'database_for_dict.dict1'; + +SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36} + +ATTACH DICTIONARY database_for_dict.dict1; + +SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); + +DROP DICTIONARY database_for_dict.dict1; + +SYSTEM RELOAD DICTIONARY 'database_for_dict.dict1'; + +SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36} + +CREATE DICTIONARY database_for_dict.dict1 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq', + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column, third_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 1)); + +SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); +SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', tuple(toUInt64(14), '196')); + +DETACH DICTIONARY database_for_dict.dict1; + +SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); -- {serverError 36} + +ATTACH DICTIONARY database_for_dict.dict1; + +SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); + +CREATE DICTIONARY database_for_dict.dict2 +( + key_column UInt64 DEFAULT 0, + some_column String EXPRESSION toString(fourth_column), + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(HASHED()); + +SELECT dictGetString('database_for_dict.dict2', 'some_column', toUInt64(12)); + +SELECT name, engine FROM system.tables WHERE database = 'database_for_dict' ORDER BY name; + +SELECT database, name, type FROM system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; + +-- check dictionary will not update +CREATE DICTIONARY database_for_dict.dict3 +( + key_column UInt64 DEFAULT 0, + some_column String EXPRESSION toString(fourth_column), + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +LIFETIME(0) +LAYOUT(HASHED()); + +SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12)); + +DROP TABLE database_for_dict.table_for_dict; + +SYSTEM RELOAD DICTIONARIES; + +SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12)); + +DROP DATABASE IF EXISTS database_for_dict; diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference new file mode 100644 index 00000000000..0c5d37b815f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_special.reference @@ -0,0 +1,11 @@ +0.33 +0.42 +0.46 +0 +17501 +NP +Moscow +[3,2,1,10000] +1 +1 +0 diff --git a/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql new file mode 100644 index 00000000000..196e78480a1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -0,0 +1,94 @@ +SET send_logs_level = 'none'; + +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict Engine = Ordinary; + +CREATE TABLE database_for_dict.table_for_dict +( + CountryID UInt64, + StartDate Date, + EndDate Date, + Tax Float64 +) +ENGINE = MergeTree() +ORDER BY CountryID; + +INSERT INTO database_for_dict.table_for_dict VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); +INSERT INTO database_for_dict.table_for_dict VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); +INSERT INTO database_for_dict.table_for_dict VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); + +CREATE DICTIONARY database_for_dict.dict1 +( + CountryID UInt64, + StartDate Date, + EndDate Date, + Tax Float64 +) +PRIMARY KEY CountryID +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(RANGE_HASHED()) +RANGE(MIN StartDate MAX EndDate); + +SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-15')); +SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-29')); +SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-29')); +SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-31')); + +CREATE TABLE database_for_dict.table_ip_trie +( + prefix String, + asn UInt32, + cca2 String +) +engine = TinyLog; + +INSERT INTO database_for_dict.table_ip_trie VALUES ('202.79.32.0/20', 17501, 'NP'), ('2620:0:870::/48', 3856, 'US'), ('2a02:6b8:1::/48', 13238, 'RU'), ('2001:db8::/32', 65536, 'ZZ'); + + +CREATE DICTIONARY database_for_dict.dict_ip_trie +( + prefix String, + asn UInt32, + cca2 String +) +PRIMARY KEY prefix +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie')) +LAYOUT(IP_TRIE()) +LIFETIME(MIN 10 MAX 100); + +SELECT dictGetUInt32('database_for_dict.dict_ip_trie', 'asn', tuple(IPv4StringToNum('202.79.32.0'))); +SELECT dictGetString('database_for_dict.dict_ip_trie', 'cca2', tuple(IPv4StringToNum('202.79.32.0'))); + +CREATE TABLE database_for_dict.table_with_hierarchy +( + RegionID UInt64, + ParentRegionID UInt64, + RegionName String +) +ENGINE = MergeTree() +ORDER BY RegionID; + +INSERT INTO database_for_dict.table_with_hierarchy VALUES (3, 2, 'Hamovniki'), (2, 1, 'Moscow'), (1, 10000, 'Russia') (7, 10000, 'Ulan-Ude'); + + +CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy +( + RegionID UInt64, + ParentRegionID UInt64 HIERARCHICAL, + RegionName String +) +PRIMARY KEY RegionID +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_with_hierarchy')) +LAYOUT(HASHED()) +LIFETIME(MIN 1 MAX 1000); + +SELECT dictGetString('database_for_dict.dictionary_with_hierarchy', 'RegionName', toUInt64(2)); +SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(3)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(3), toUInt64(2)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(7), toUInt64(10000)); +SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(1), toUInt64(5)); + +DROP DATABASE IF EXISTS database_for_dict; + diff --git a/dbms/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.reference b/dbms/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.reference new file mode 100644 index 00000000000..87dc6a5b6bf --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.reference @@ -0,0 +1,16 @@ +100 +100 +200 +200 +200 +300 +300 +300 +dict1 +dict2 +dict3 +table_for_dict +dict1 +dict2 +dict3 +dict4 diff --git a/dbms/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/dbms/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql new file mode 100644 index 00000000000..029dc9755d8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -0,0 +1,100 @@ +SET send_logs_level = 'none'; + +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict Engine = Ordinary; + +CREATE TABLE database_for_dict.table_for_dict +( + key_column UInt64, + second_column UInt8, + third_column String, + fourth_column Float64 +) +ENGINE = MergeTree() +ORDER BY key_column; + +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); + +DROP DICTIONARY IF EXISTS database_for_dict.dict1; + +CREATE DICTIONARY database_for_dict.dict1 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq', + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +SELECT count(*) from database_for_dict.dict1; + +CREATE DICTIONARY database_for_dict.dict2 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq', + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict1' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(HASHED()); + +SELECT count(*) FROM database_for_dict.dict2; + +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); + +SYSTEM RELOAD DICTIONARIES; + +SELECT count(*) from database_for_dict.dict2; +SELECT count(*) from database_for_dict.dict1; + +CREATE DICTIONARY database_for_dict.dict3 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq', + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict2' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(HASHED()); + +SELECT count(*) FROM database_for_dict.dict3; + +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); + +SYSTEM RELOAD DICTIONARIES; + +SELECT count(*) from database_for_dict.dict3; +SELECT count(*) from database_for_dict.dict2; +SELECT count(*) from database_for_dict.dict1; + + +CREATE DICTIONARY database_for_dict.dict4 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq', + fourth_column Float64 DEFAULT 42.0 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'non_existing_table' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(HASHED()); + +SELECT count(*) FROM database_for_dict.dict4; -- {serverError 60} + +SELECT name from system.tables WHERE database = 'database_for_dict' ORDER BY name; +SELECT name from system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; + +DROP DATABASE IF EXISTS database_for_dict; + +SELECT count(*) from database_for_dict.dict3; --{serverError 81} +SELECT count(*) from database_for_dict.dict2; --{serverError 81} +SELECT count(*) from database_for_dict.dict1; --{serverError 81} diff --git a/dbms/tests/queries/0_stateless/01020_having_without_group_by.reference b/dbms/tests/queries/0_stateless/01020_having_without_group_by.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01020_having_without_group_by.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/01020_having_without_group_by.sql b/dbms/tests/queries/0_stateless/01020_having_without_group_by.sql new file mode 100644 index 00000000000..cf9b59b35bd --- /dev/null +++ b/dbms/tests/queries/0_stateless/01020_having_without_group_by.sql @@ -0,0 +1 @@ +SELECT 1 HAVING 1; diff --git a/dbms/tests/server-test.xml b/dbms/tests/server-test.xml index d68cbca53c1..d9e547b4d55 100644 --- a/dbms/tests/server-test.xml +++ b/dbms/tests/server-test.xml @@ -75,6 +75,20 @@
+ + + + 127.0.0.1 + 59000 + + + + + 127.0.0.2 + 59000 + + + diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index d88a2767efd..c15de71e848 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -57,20 +57,28 @@ RUN apt-get update -y \ rename \ wget -# Build and install tools for cross-linking to Darwin - ENV CC=clang-8 ENV CXX=clang++-8 # libtapi is required to support .tbh format from recent MacOS SDKs RUN git clone https://github.com/tpoechtrager/apple-libtapi.git RUN cd apple-libtapi && INSTALLPREFIX=/cctools ./build.sh && ./install.sh +RUN rm -rf apple-libtapi +# Build and install tools for cross-linking to Darwin RUN git clone https://github.com/tpoechtrager/cctools-port.git RUN cd cctools-port/cctools && ./configure --prefix=/cctools --with-libtapi=/cctools --target=x86_64-apple-darwin && make install +RUN rm -rf cctools-port +# Download toolchain for Darwin +RUN mkdir -p /build/cmake/toolchain/darwin-x86_64 RUN wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz -RUN tar xJf MacOSX10.14.sdk.tar.xz -C /cctools +RUN tar xJf MacOSX10.14.sdk.tar.xz -C /build/cmake/toolchain/darwin-x86_64 --strip-components=1 + +# Download toolchain for ARM +RUN mkdir -p /build/cmake/toolchain/linux-aarch64 +RUN wget "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en" -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz +RUN tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -C /build/cmake/toolchain/linux-aarch64 --strip-components=1 COPY build.sh / CMD ["/bin/bash", "/build.sh"] diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 436e0bdad8a..56151911f50 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -366,12 +366,15 @@ For more information, see the section "[Creating replicated tables](../../operat ``` -## mark_cache_size +## mark_cache_size {#server-mark-cache-size} Approximate size (in bytes) of the cache of "marks" used by [MergeTree](../../operations/table_engines/mergetree.md). The cache is shared for the server and memory is allocated as needed. The cache size must be at least 5368709120. +!!! note IMPORTANT + This parameter could be exceeded by user's setting [mark_cache_min_lifetime](../settings/settings.md#settings-mark_cache_min_lifetime). + **Example** ```xml diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 53b50931ec2..159d0cbe7ff 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -570,6 +570,12 @@ We are writing a URL column with the String type (average size of 60 bytes per v There usually isn't any reason to change this setting. +## mark_cache_min_lifetime {#settings-mark_cache_min_lifetime} + +If the value of [mark_cache_size](../server_settings/settings.md#server-mark-cache-size) setting is exceeded, delete only records older than mark_cache_min_lifetime seconds. If your hosts have low amount of RAM, it makes sense to lower this parameter. + +Default value: 10000 seconds. + ## max_query_size {#settings-max_query_size} The maximum part of a query that can be taken to RAM for parsing with the SQL parser. diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index b7b37924c71..f275a908fd9 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -294,7 +294,7 @@ This query creates a local backup of a specified partition. If the `PARTITION` c Note that for old-styled tables you can specify the prefix of the partition name (for example, '2019') - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). !!! note - The entire backup process is performed without stopping the server. + The entire backup process is performed without stopping the server. At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where: @@ -389,7 +389,7 @@ Possible values: `0` – do not wait; `1` – only wait for own execution (defau Mutations are an ALTER query variant that allows changing or deleting rows in a table. In contrast to standard `UPDATE` and `DELETE` queries that are intended for point data changes, mutations are intended for heavy operations that change a lot of rows in a table. -The functionality is in beta stage and is available starting with the 1.1.54388 version. Currently `*MergeTree` table engines are supported (both replicated and unreplicated). +Currently `*MergeTree` table engines are supported (both replicated and unreplicated). Existing tables are ready for mutations as-is (no conversion necessary), but after the first mutation is applied to a table, its metadata format becomes incompatible with previous server versions and falling back to a previous version becomes impossible. @@ -405,7 +405,7 @@ The `filter_expr` must be of type UInt8. The query deletes rows in the table for ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` -The command is available starting with the 18.12.14 version. The `filter_expr` must be of type UInt8. This query updates values of specified columns to the values of corresponding expressions in rows for which the `filter_expr` takes a non-zero value. Values are casted to the column type using the `CAST` operator. Updating columns that are used in the calculation of the primary or the partition key is not supported. +The `filter_expr` must be of type UInt8. This query updates values of specified columns to the values of corresponding expressions in rows for which the `filter_expr` takes a non-zero value. Values are casted to the column type using the `CAST` operator. Updating columns that are used in the calculation of the primary or the partition key is not supported. ```sql ALTER TABLE [db.]table MATERIALIZE INDEX name IN PARTITION partition_name diff --git a/docs/en/query_language/functions/string_functions.md b/docs/en/query_language/functions/string_functions.md index 02a8e1d64aa..32186bfb74e 100644 --- a/docs/en/query_language/functions/string_functions.md +++ b/docs/en/query_language/functions/string_functions.md @@ -195,7 +195,20 @@ Returns a string that removes the whitespace characters on either side. ## CRC32(s) -Returns the CRC32 checksum of a string +Returns the CRC32 checksum of a string, using CRC-32-IEEE 802.3 polynomial and initial value `0xffffffff` (zlib implementation). + The result type is UInt32. +## CRC32IEEE(s) + +Returns the CRC32 checksum of a string, using CRC-32-IEEE 802.3 polynomial. + +The result type is UInt32. + +## CRC64(s) + +Returns the CRC64 checksum of a string, using CRC-64-ECMA polynomial. + +The result type is UInt64. + [Original article](https://clickhouse.yandex/docs/en/query_language/functions/string_functions/) diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index 22d67044619..9bcac5cdbfd 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -174,7 +174,7 @@ Changes already made by the mutation are not rolled back. ## OPTIMIZE {#misc_operations-optimize} ```sql -OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL] +OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] ``` This query tries to initialize an unscheduled merge of data parts for tables with a table engine from the [MergeTree](../operations/table_engines/mergetree.md) family. Other kinds of table engines aren't supported. @@ -182,7 +182,7 @@ This query tries to initialize an unscheduled merge of data parts for tables wit When `OPTIMIZE` is used with the [ReplicatedMergeTree](../operations/table_engines/replication.md) family of table engines, ClickHouse creates a task for merging and waits for execution on all nodes (if the `replication_alter_partitions_sync` setting is enabled). - If `OPTIMIZE` doesn't perform a merge for any reason, it doesn't notify the client. To enable notifications, use the [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop) setting. -- If you specify a `PARTITION`, only the specified partition is optimized. +- If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](alter.md#alter-how-to-specify-part-expr). - If you specify `FINAL`, optimization is performed even when all the data is already in one part. !!! warning "Warning" diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index e72d97bdc46..f95b0809650 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -366,12 +366,15 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ``` -## mark_cache_size +## mark_cache_size {#server-mark-cache-size} Приблизительный размер (в байтах) кеша "засечек", используемых движками таблиц семейства [MergeTree](../../operations/table_engines/mergetree.md). Кеш общий для сервера, память выделяется по мере необходимости. Кеш не может быть меньше, чем 5368709120. +!!! note ВАЖНО + Этот параметр может быть превышен при большом значении настройки пользователя [mark_cache_min_lifetime](../settings/settings.md#settings-mark_cache_min_lifetime). + **Пример** ```xml diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index ba4f07cda3b..d37e0911698 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -511,6 +511,12 @@ ClickHouse использует этот параметр при чтении д Как правило, не имеет смысла менять эту настройку. +## mark_cache_min_lifetime {#settings-mark_cache_min_lifetime} + +Если превышено значение параметра [mark_cache_size](../server_settings/settings.md#server-mark-cache-size), то будут удалены только записи старше чем значение этого параметра. Имеет смысл понижать данный параметр при малом количестве RAM на хост-системах. + +Default value: 10000 seconds. + ## max_query_size {#settings-max_query_size} Максимальный кусок запроса, который будет считан в оперативку для разбора парсером языка SQL. diff --git a/docs/ru/query_language/functions/string_functions.md b/docs/ru/query_language/functions/string_functions.md index 193da6f2753..e6753247ea9 100644 --- a/docs/ru/query_language/functions/string_functions.md +++ b/docs/ru/query_language/functions/string_functions.md @@ -155,7 +155,20 @@ SELECT startsWith('Hello, world!', 'He'); ## CRC32(s) -Возвращает чексумму CRC32 данной строки. +Возвращает чексумму CRC32 данной строки, используется CRC-32-IEEE 802.3 многочлен и начальным значением `0xffffffff` (т.к. используется реализация из zlib). + Тип результата - UInt32. +## CRC32IEEE(s) + +Возвращает чексумму CRC32 данной строки, используется CRC-32-IEEE 802.3 многочлен. + +Тип результата - UInt32. + +## CRC64(s) + +Возвращает чексумму CRC64 данной строки, используется CRC-64-ECMA многочлен. + +Тип результата - UInt64. + [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/string_functions/) diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 00cb0e7fd93..ce73a5aafdb 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -173,7 +173,7 @@ KILL MUTATION WHERE database = 'default' AND table = 'table' AND mutation_id = ' ## OPTIMIZE {#misc_operations-optimize} ```sql -OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL] +OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition | PARTITION ID 'partition_id'] [FINAL] ``` Запрос пытается запустить внеплановый мёрж кусков данных для таблиц семейства [MergeTree](../operations/table_engines/mergetree.md). Другие движки таблиц не поддерживаются. @@ -181,7 +181,7 @@ OPTIMIZE TABLE [db.]name [ON CLUSTER cluster] [PARTITION partition] [FINAL] Если `OPTIMIZE` применяется к таблицам семейства [ReplicatedMergeTree](../operations/table_engines/replication.md), ClickHouse создаёт задачу на мёрж и ожидает её исполнения на всех узлах (если активирована настройка `replication_alter_partitions_sync`). - Если `OPTIMIZE` не выполняет мёрж по любой причине, ClickHouse не оповещает об этом клиента. Чтобы включить оповещения, используйте настройку [optimize_throw_if_noop](../operations/settings/settings.md#setting-optimize_throw_if_noop). -- Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. +- Если указать `PARTITION`, то оптимизация выполняется только для указанной партиции. [Как задавать имя партиции в запросах](alter.md#alter-how-to-specify-part-expr). - Если указать `FINAL`, то оптимизация выполняется даже в том случае, если все данные уже лежат в одном куске. !!! warning "Внимание" diff --git a/docs/zh/data_types/fixedstring.md b/docs/zh/data_types/fixedstring.md index 050f0a582a2..18ed3ed45bc 100644 --- a/docs/zh/data_types/fixedstring.md +++ b/docs/zh/data_types/fixedstring.md @@ -1,9 +1,55 @@ -# FixedString(N) +# FixedString -固定长度 N 的字符串。N 必须是严格的正自然数。 -当服务端读取长度小于 N 的字符串时候(例如解析 INSERT 数据时),通过在字符串末尾添加空字节来达到 N 字节长度。 -当服务端读取长度大于 N 的字符串时候,将返回错误消息。 -当服务器写入一个字符串(例如,当输出 SELECT 查询的结果)时,NULL字节不会从字符串的末尾被移除,而是被输出。 -注意这种方式与 MYSQL 的 CHAR 类型是不一样的(MYSQL 的字符串会以空格填充,然后输出的时候空格会被修剪)。 +固定长度 N 的字符串(N 必须是严格的正自然数)。 -与 `String` 类型相比,极少的函数会使用 `FixedString(N)`,因此使用起来不太方便。 +您可以使用下面的语法对列声明为`FixedString`类型: + +```sql + FixedString(N) +``` + +其中`N`表示自然数。 + +当数据的长度恰好为N个字节时,`FixedString`类型是高效的。 在其他情况下,这可能会降低效率。 + +可以有效存储在`FixedString`类型的列中的值的示例: + +- 二进制表示的IP地址(IPv6使用`FixedString(16)`) +- 语言代码(ru_RU, en_US ... ) +- 货币代码(USD, RUB ... ) +- 二进制表示的哈希值(MD5使用`FixedString(16)`,SHA256使用`FixedString(32)`) + +请使用[UUID](uuid.md)数据类型来存储UUID值,。 + +当向ClickHouse中插入数据时, + +- 如果字符串包含的字节数少于`N',将对字符串末尾进行空字节填充。 +- 如果字符串包含的字节数大于`N`,将抛出`Too large value for FixedString(N)`异常。 + +当做数据查询时,ClickHouse不会删除字符串末尾的空字节。 如果使用`WHERE`子句,则须要手动添加空字节以匹配`FixedString`的值。 以下示例阐明了如何将`WHERE`子句与`FixedString`一起使用。 + +考虑带有`FixedString(2)`列的表: + +```text +┌─name──┐ +│ b │ +└───────┘ +``` + +查询语句`SELECT * FROM FixedStringTable WHERE a = 'b'` 不会返回任何结果。请使用空字节来填充筛选条件。 + +```sql +SELECT * FROM FixedStringTable +WHERE a = 'b\0' +``` +```text +┌─a─┐ +│ b │ +└───┘ +``` + +这种方式与MySQL的`CHAR`类型的方式不同(MySQL中使用空格填充字符串,并在输出时删除空格)。 + +请注意,`FixedString(N)`的长度是个常量。仅由空字符组成的字符串,函数[length](../query_language/functions/array_functions.md#array_functions-length)返回值为`N`,而函数[empty](../query_language/functions/string_functions.md#string_functions-empty)的返回值为`1`。 + +[来源文章](https://clickhouse.yandex/docs/en/data_types/fixedstring/) diff --git a/docs/zh/data_types/nullable.md b/docs/zh/data_types/nullable.md index 41565f9d721..67d98fa9408 100644 --- a/docs/zh/data_types/nullable.md +++ b/docs/zh/data_types/nullable.md @@ -19,37 +19,21 @@ ## 用法示例 + +```sql +CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog ``` -:) CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE TinyLog - -CREATE TABLE t_null -( - x Int8, - y Nullable(Int8) -) -ENGINE = TinyLog - -Ok. - -0 rows in set. Elapsed: 0.012 sec. - -:) INSERT INTO t_null VALUES (1, NULL) - -INSERT INTO t_null VALUES - -Ok. - -1 rows in set. Elapsed: 0.007 sec. - -:) SELECT x + y FROM t_null - -SELECT x + y -FROM t_null - +```sql +INSERT INTO t_null VALUES (1, NULL), (2, 3) +``` +```sql +SELECT x + y FROM t_null +``` +```text ┌─plus(x, y)─┐ │ ᴺᵁᴸᴸ │ │ 5 │ └────────────┘ - -2 rows in set. Elapsed: 0.144 sec. ``` + +[来源文章](https://clickhouse.yandex/docs/en/data_types/nullable/) \ No newline at end of file