mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 21:24:28 +00:00
Merge branch 'master' into database_atomic
This commit is contained in:
commit
5059c78e81
3
.github/CODEOWNERS
vendored
Normal file
3
.github/CODEOWNERS
vendored
Normal file
@ -0,0 +1,3 @@
|
||||
dbms/* @ClickHouse/core-assigner
|
||||
docs/* @ClickHouse/docs
|
||||
docs/zh/* @ClickHouse/docs-zh
|
1
.github/labeler.keywords.yml
vendored
1
.github/labeler.keywords.yml
vendored
@ -1 +0,0 @@
|
||||
pr-feature: "New Feature"
|
23
.github/labeler.yml
vendored
23
.github/labeler.yml
vendored
@ -1,23 +0,0 @@
|
||||
# Build changes
|
||||
pr-build:
|
||||
- "**/CMakeLists.txt"
|
||||
|
||||
# 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/**/*"
|
11
.github/workflows/labeler.yml
vendored
11
.github/workflows/labeler.yml
vendored
@ -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 }}"
|
@ -1,13 +1,20 @@
|
||||
macro(add_glob cur_list)
|
||||
file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${ARGN})
|
||||
list(APPEND ${cur_list} ${__tmp})
|
||||
endmacro()
|
||||
if (CMAKE_VERSION VERSION_GREATER_EQUAL "3.12")
|
||||
macro(add_glob cur_list)
|
||||
file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} CONFIGURE_DEPENDS ${ARGN})
|
||||
list(APPEND ${cur_list} ${__tmp})
|
||||
endmacro()
|
||||
else ()
|
||||
macro(add_glob cur_list)
|
||||
file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${ARGN})
|
||||
list(APPEND ${cur_list} ${__tmp})
|
||||
endmacro()
|
||||
endif ()
|
||||
|
||||
macro(add_headers_and_sources prefix common_path)
|
||||
add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
|
||||
add_glob(${prefix}_headers ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
|
||||
add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h)
|
||||
endmacro()
|
||||
|
||||
macro(add_headers_only prefix common_path)
|
||||
add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
|
||||
add_glob(${prefix}_headers ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
|
||||
endmacro()
|
||||
|
10
contrib/CMakeLists.txt
vendored
10
contrib/CMakeLists.txt
vendored
@ -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)
|
||||
|
@ -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)
|
||||
@ -153,12 +153,10 @@ add_subdirectory(src/Common/Config)
|
||||
set (all_modules)
|
||||
macro(add_object_library name common_path)
|
||||
if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES)
|
||||
add_glob(dbms_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
|
||||
add_glob(dbms_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h)
|
||||
add_headers_and_sources(dbms ${common_path})
|
||||
else ()
|
||||
list (APPEND all_modules ${name})
|
||||
add_glob(${name}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
|
||||
add_glob(${name}_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h)
|
||||
add_headers_and_sources(${name} ${common_path})
|
||||
add_library(${name} SHARED ${${name}_sources} ${${name}_headers})
|
||||
target_link_libraries (${name} PRIVATE -Wl,--unresolved-symbols=ignore-all)
|
||||
endif ()
|
||||
|
@ -1,11 +1,11 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_REVISION 54427)
|
||||
set(VERSION_REVISION 54428)
|
||||
set(VERSION_MAJOR 19)
|
||||
set(VERSION_MINOR 16)
|
||||
set(VERSION_MINOR 17)
|
||||
set(VERSION_PATCH 1)
|
||||
set(VERSION_GITHASH 38f65a6a2120d2e76bcf71131068f41195149dfc)
|
||||
set(VERSION_DESCRIBE v19.16.1.1-prestable)
|
||||
set(VERSION_STRING 19.16.1.1)
|
||||
set(VERSION_GITHASH 5286d0afb285a5fbf3d320af3daa6de6b1841374)
|
||||
set(VERSION_DESCRIBE v19.17.1.1-prestable)
|
||||
set(VERSION_STRING 19.17.1.1)
|
||||
# end of autochange
|
||||
|
||||
set(VERSION_EXTRA "" CACHE STRING "")
|
||||
|
@ -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); });
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <Poco/Net/HTTPServerResponse.h>
|
||||
#include <Poco/Net/HTMLForm.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <IO/WriteBufferFromHTTPServerResponse.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Parsers/ParserQueryWithOutput.h>
|
||||
@ -37,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:
|
||||
@ -154,9 +159,19 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
|
||||
SQLSMALLINT type = 0;
|
||||
/// TODO Why 301?
|
||||
SQLCHAR column_name[301];
|
||||
/// TODO Result is not checked.
|
||||
POCO_SQL_ODBC_CLASS::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, nullptr);
|
||||
columns.emplace_back(reinterpret_cast<char *>(column_name), getDataType(type));
|
||||
|
||||
SQLSMALLINT nullable;
|
||||
const auto result = POCO_SQL_ODBC_CLASS::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, &nullable);
|
||||
if (POCO_SQL_ODBC_CLASS::Utility::isError(result))
|
||||
throw POCO_SQL_ODBC_CLASS::StatementException(hstmt);
|
||||
|
||||
auto column_type = getDataType(type);
|
||||
if (nullable == SQL_NULLABLE)
|
||||
{
|
||||
column_type = std::make_shared<DataTypeNullable>(column_type);
|
||||
}
|
||||
|
||||
columns.emplace_back(reinterpret_cast<char *>(column_name), std::move(column_type));
|
||||
}
|
||||
|
||||
WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout);
|
||||
|
@ -44,7 +44,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
|
||||
for (auto iterator = db.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
{
|
||||
auto & table = iterator->table();
|
||||
StorageReplicatedMergeTree * table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
|
||||
|
@ -37,10 +37,12 @@
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
#include <Interpreters/DDLWorker.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/ExternalModelsLoader.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
#include <Interpreters/DNSCacheUpdater.h>
|
||||
#include <Interpreters/SystemLog.cpp>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
@ -920,6 +922,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
global_context->tryCreateEmbeddedDictionaries();
|
||||
global_context->getExternalDictionariesLoader().enableAlwaysLoadEverything(true);
|
||||
}
|
||||
|
||||
auto dictionaries_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "dictionaries_config");
|
||||
global_context->getExternalDictionariesLoader().addConfigRepository("", std::move(dictionaries_repository));
|
||||
|
||||
auto models_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config(), "models_config");
|
||||
global_context->getExternalModelsLoader().addConfigRepository("", std::move(models_repository));
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -153,7 +153,7 @@ private:
|
||||
void receiveHello();
|
||||
bool receivePacket();
|
||||
void receiveQuery();
|
||||
bool receiveData();
|
||||
bool receiveData(bool scalar);
|
||||
bool readDataNext(const size_t & poll_interval, const int & receive_timeout);
|
||||
void readData(const Settings & global_settings);
|
||||
std::tuple<size_t, int> getReadTimeouts(const Settings & global_settings);
|
||||
|
@ -180,7 +180,21 @@
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_shards_localhost>
|
||||
</test_cluster_two_shards_localhost>
|
||||
<test_cluster_two_shards>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>127.0.0.1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>127.0.0.2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_shards>
|
||||
<test_shard_localhost_secure>
|
||||
<shard>
|
||||
<replica>
|
||||
|
@ -31,9 +31,9 @@ struct AggregateFunctionAvgData
|
||||
if constexpr (std::numeric_limits<ResultT>::is_iec559)
|
||||
return static_cast<ResultT>(sum) / count; /// allow division by zero
|
||||
|
||||
if (!count)
|
||||
throw Exception("AggregateFunctionAvg with zero values", ErrorCodes::LOGICAL_ERROR);
|
||||
return static_cast<ResultT>(sum) / count;
|
||||
if (count == 0)
|
||||
return static_cast<ResultT>(0);
|
||||
return static_cast<ResultT>(sum / count);
|
||||
}
|
||||
};
|
||||
|
||||
@ -43,10 +43,10 @@ template <typename T, typename Data>
|
||||
class AggregateFunctionAvg final : public IAggregateFunctionDataHelper<Data, AggregateFunctionAvg<T, Data>>
|
||||
{
|
||||
public:
|
||||
using ResultType = std::conditional_t<IsDecimalNumber<T>, Decimal128, Float64>;
|
||||
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<Decimal128>, DataTypeNumber<Float64>>;
|
||||
using ResultType = std::conditional_t<IsDecimalNumber<T>, T, Float64>;
|
||||
using ResultDataType = std::conditional_t<IsDecimalNumber<T>, DataTypeDecimal<T>, DataTypeNumber<Float64>>;
|
||||
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
|
||||
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<Decimal128>, ColumnVector<Float64>>;
|
||||
using ColVecResult = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<Float64>>;
|
||||
|
||||
/// ctor for native types
|
||||
AggregateFunctionAvg(const DataTypes & argument_types_)
|
||||
|
@ -109,8 +109,11 @@ public:
|
||||
double quantileInterpolated(double level)
|
||||
{
|
||||
if (samples.empty())
|
||||
{
|
||||
if (DB::IsDecimalNumber<T>)
|
||||
return 0;
|
||||
return onEmpty<double>();
|
||||
|
||||
}
|
||||
sortIfNeeded();
|
||||
|
||||
double index = std::max(0., std::min(samples.size() - 1., level * (samples.size() - 1)));
|
||||
|
@ -30,6 +30,7 @@
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric SendScalars;
|
||||
extern const Metric SendExternalTables;
|
||||
}
|
||||
|
||||
@ -441,7 +442,7 @@ void Connection::sendCancel()
|
||||
}
|
||||
|
||||
|
||||
void Connection::sendData(const Block & block, const String & name)
|
||||
void Connection::sendData(const Block & block, const String & name, bool scalar)
|
||||
{
|
||||
//LOG_TRACE(log_wrapper.get(), "Sending data");
|
||||
|
||||
@ -455,7 +456,10 @@ void Connection::sendData(const Block & block, const String & name)
|
||||
block_out = std::make_shared<NativeBlockOutputStream>(*maybe_compressed_out, server_revision, block.cloneEmpty());
|
||||
}
|
||||
|
||||
writeVarUInt(Protocol::Client::Data, *out);
|
||||
if (scalar)
|
||||
writeVarUInt(Protocol::Client::Scalar, *out);
|
||||
else
|
||||
writeVarUInt(Protocol::Client::Data, *out);
|
||||
writeStringBinary(name, *out);
|
||||
|
||||
size_t prev_bytes = out->count();
|
||||
@ -484,6 +488,44 @@ void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String
|
||||
}
|
||||
|
||||
|
||||
void Connection::sendScalarsData(Scalars & data)
|
||||
{
|
||||
if (data.empty())
|
||||
return;
|
||||
|
||||
Stopwatch watch;
|
||||
size_t out_bytes = out ? out->count() : 0;
|
||||
size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0;
|
||||
size_t rows = 0;
|
||||
|
||||
CurrentMetrics::Increment metric_increment{CurrentMetrics::SendScalars};
|
||||
|
||||
for (auto & elem : data)
|
||||
{
|
||||
rows += elem.second.rows();
|
||||
sendData(elem.second, elem.first, true /* scalar */);
|
||||
}
|
||||
|
||||
out_bytes = out->count() - out_bytes;
|
||||
maybe_compressed_out_bytes = maybe_compressed_out->count() - maybe_compressed_out_bytes;
|
||||
double elapsed = watch.elapsedSeconds();
|
||||
|
||||
std::stringstream msg;
|
||||
msg << std::fixed << std::setprecision(3);
|
||||
msg << "Sent data for " << data.size() << " scalars, total " << rows << " rows in " << elapsed << " sec., "
|
||||
<< static_cast<size_t>(rows / watch.elapsedSeconds()) << " rows/sec., "
|
||||
<< maybe_compressed_out_bytes / 1048576.0 << " MiB (" << maybe_compressed_out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
|
||||
|
||||
if (compression == Protocol::Compression::Enable)
|
||||
msg << ", compressed " << static_cast<double>(maybe_compressed_out_bytes) / out_bytes << " times to "
|
||||
<< out_bytes / 1048576.0 << " MiB (" << out_bytes / 1048576.0 / watch.elapsedSeconds() << " MiB/sec.)";
|
||||
else
|
||||
msg << ", no compression.";
|
||||
|
||||
LOG_DEBUG(log_wrapper.get(), msg.rdbuf());
|
||||
}
|
||||
|
||||
|
||||
void Connection::sendExternalTablesData(ExternalTablesData & data)
|
||||
{
|
||||
if (data.empty())
|
||||
|
@ -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);
|
||||
|
||||
|
@ -51,6 +51,21 @@ MultiplexedConnections::MultiplexedConnections(
|
||||
active_connection_count = connections.size();
|
||||
}
|
||||
|
||||
void MultiplexedConnections::sendScalarsData(Scalars & data)
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
||||
if (!sent_query)
|
||||
throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (ReplicaState & state : replica_states)
|
||||
{
|
||||
Connection * connection = state.connection;
|
||||
if (connection != nullptr)
|
||||
connection->sendScalarsData(data);
|
||||
}
|
||||
}
|
||||
|
||||
void MultiplexedConnections::sendExternalTablesData(std::vector<ExternalTablesData> & data)
|
||||
{
|
||||
std::lock_guard lock(cancel_mutex);
|
||||
|
@ -27,6 +27,8 @@ public:
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
const Settings & settings_, const ThrottlerPtr & throttler_);
|
||||
|
||||
/// Send all scalars to replicas.
|
||||
void sendScalarsData(Scalars & data);
|
||||
/// Send all content of external tables to replicas.
|
||||
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
|
||||
|
||||
|
@ -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.") \
|
||||
|
@ -292,7 +292,7 @@ Volume::Volume(
|
||||
formatReadableSizeWithBinarySuffix(max_data_part_size) << ")");
|
||||
}
|
||||
constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u;
|
||||
if (max_data_part_size < MIN_PART_SIZE)
|
||||
if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE)
|
||||
LOG_WARNING(logger, "Volume " << backQuote(name) << " max_data_part_size is too low ("
|
||||
<< formatReadableSizeWithBinarySuffix(max_data_part_size) << " < "
|
||||
<< formatReadableSizeWithBinarySuffix(MIN_PART_SIZE) << ")");
|
||||
|
@ -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;
|
||||
|
@ -358,6 +358,12 @@ protected:
|
||||
template <typename, typename, typename, typename, typename, typename, size_t>
|
||||
friend class TwoLevelHashTable;
|
||||
|
||||
template <typename, typename, size_t>
|
||||
friend class TwoLevelStringHashTable;
|
||||
|
||||
template <typename SubMaps>
|
||||
friend class StringHashTable;
|
||||
|
||||
using HashValue = size_t;
|
||||
using Self = HashTable;
|
||||
using cell_type = Cell;
|
||||
|
180
dbms/src/Common/HashTable/StringHashMap.h
Normal file
180
dbms/src/Common/HashTable/StringHashMap.h
Normal file
@ -0,0 +1,180 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashTableAllocator.h>
|
||||
#include <Common/HashTable/StringHashTable.h>
|
||||
|
||||
template <typename Key, typename TMapped>
|
||||
struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashMapCell<Key, TMapped, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
};
|
||||
|
||||
template<typename Key, typename Mapped>
|
||||
auto lookupResultGetMapped(StringHashMapCell<Key, Mapped> * cell) { return &cell->getSecond(); }
|
||||
|
||||
template <typename TMapped>
|
||||
struct StringHashMapCell<StringKey16, TMapped> : public HashMapCell<StringKey16, TMapped, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashMapCell<StringKey16, TMapped, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
bool isZero(const HashTableNoState & state) const { return isZero(this->value.first, state); }
|
||||
// Assuming String does not contain zero bytes. NOTE: Cannot be used in serialized method
|
||||
static bool isZero(const StringKey16 & key, const HashTableNoState & /*state*/) { return key.low == 0; }
|
||||
void setZero() { this->value.first.low = 0; }
|
||||
};
|
||||
|
||||
template <typename TMapped>
|
||||
struct StringHashMapCell<StringKey24, TMapped> : public HashMapCell<StringKey24, TMapped, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashMapCell<StringKey24, TMapped, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
bool isZero(const HashTableNoState & state) const { return isZero(this->value.first, state); }
|
||||
// Assuming String does not contain zero bytes. NOTE: Cannot be used in serialized method
|
||||
static bool isZero(const StringKey24 & key, const HashTableNoState & /*state*/) { return key.a == 0; }
|
||||
void setZero() { this->value.first.a = 0; }
|
||||
};
|
||||
|
||||
template <typename TMapped>
|
||||
struct StringHashMapCell<StringRef, TMapped> : public HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash, HashTableNoState>
|
||||
{
|
||||
using Base = HashMapCellWithSavedHash<StringRef, TMapped, StringHashTableHash, HashTableNoState>;
|
||||
using Base::Base;
|
||||
static constexpr bool need_zero_value_storage = false;
|
||||
};
|
||||
|
||||
template <typename TMapped, typename Allocator>
|
||||
struct StringHashMapSubMaps
|
||||
{
|
||||
using T0 = StringHashTableEmpty<StringHashMapCell<StringRef, TMapped>>;
|
||||
using T1 = HashMapTable<StringKey8, StringHashMapCell<StringKey8, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
using T2 = HashMapTable<StringKey16, StringHashMapCell<StringKey16, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
using T3 = HashMapTable<StringKey24, StringHashMapCell<StringKey24, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
using Ts = HashMapTable<StringRef, StringHashMapCell<StringRef, TMapped>, StringHashTableHash, StringHashTableGrower<>, Allocator>;
|
||||
};
|
||||
|
||||
template <typename TMapped, typename Allocator = HashTableAllocator>
|
||||
class StringHashMap : public StringHashTable<StringHashMapSubMaps<TMapped, Allocator>>
|
||||
{
|
||||
public:
|
||||
using Base = StringHashTable<StringHashMapSubMaps<TMapped, Allocator>>;
|
||||
using Self = StringHashMap;
|
||||
using Key = StringRef;
|
||||
using key_type = StringRef;
|
||||
using mapped_type = TMapped;
|
||||
using value_type = typename Base::Ts::value_type;
|
||||
using LookupResult = mapped_type *;
|
||||
|
||||
using Base::Base;
|
||||
|
||||
/// Merge every cell's value of current map into the destination map.
|
||||
/// Func should have signature void(Mapped & dst, Mapped & src, bool emplaced).
|
||||
/// Each filled cell in current map will invoke func once. If that map doesn't
|
||||
/// have a key equals to the given cell, a new cell gets emplaced into that map,
|
||||
/// and func is invoked with the third argument emplaced set to true. Otherwise
|
||||
/// emplaced is set to false.
|
||||
template <typename Func>
|
||||
void ALWAYS_INLINE mergeToViaEmplace(Self & that, Func && func)
|
||||
{
|
||||
if (this->m0.hasZero())
|
||||
{
|
||||
const bool emplace_new_zero = !that.m0.hasZero();
|
||||
if (emplace_new_zero)
|
||||
{
|
||||
that.m0.setHasZero();
|
||||
}
|
||||
|
||||
func(that.m0.zeroValue()->getSecond(), this->m0.zeroValue()->getSecond(),
|
||||
emplace_new_zero);
|
||||
}
|
||||
|
||||
this->m1.mergeToViaEmplace(that.m1, func);
|
||||
this->m2.mergeToViaEmplace(that.m2, func);
|
||||
this->m3.mergeToViaEmplace(that.m3, func);
|
||||
this->ms.mergeToViaEmplace(that.ms, func);
|
||||
}
|
||||
|
||||
/// Merge every cell's value of current map into the destination map via find.
|
||||
/// Func should have signature void(Mapped & dst, Mapped & src, bool exist).
|
||||
/// Each filled cell in current map will invoke func once. If that map doesn't
|
||||
/// have a key equals to the given cell, func is invoked with the third argument
|
||||
/// exist set to false. Otherwise exist is set to true.
|
||||
template <typename Func>
|
||||
void ALWAYS_INLINE mergeToViaFind(Self & that, Func && func)
|
||||
{
|
||||
if (this->m0.hasZero())
|
||||
{
|
||||
if (that.m0.hasZero())
|
||||
{
|
||||
func(that.m0.zeroValue()->getSecond(), this->m0.zeroValue()->getSecond(), true);
|
||||
}
|
||||
else
|
||||
{
|
||||
func(this->m0.zeroValue()->getSecond(), this->m0.zeroValue()->getSecond(), false);
|
||||
}
|
||||
}
|
||||
|
||||
this->m1.mergeToViaFind(that.m1, func);
|
||||
this->m2.mergeToViaFind(that.m2, func);
|
||||
this->m3.mergeToViaFind(that.m3, func);
|
||||
this->ms.mergeToViaFind(that.ms, func);
|
||||
}
|
||||
|
||||
mapped_type & ALWAYS_INLINE operator[](Key x)
|
||||
{
|
||||
bool inserted;
|
||||
LookupResult it = nullptr;
|
||||
emplace(x, it, inserted);
|
||||
if (inserted)
|
||||
new (it) mapped_type();
|
||||
return *it;
|
||||
}
|
||||
|
||||
template <typename Func>
|
||||
void ALWAYS_INLINE forEachValue(Func && func)
|
||||
{
|
||||
if (this->m0.size())
|
||||
{
|
||||
func(StringRef{}, this->m0.zeroValue()->getSecond());
|
||||
}
|
||||
|
||||
for (auto & v : this->m1)
|
||||
{
|
||||
func(toStringRef(v.getFirst()), v.getSecond());
|
||||
}
|
||||
|
||||
for (auto & v : this->m2)
|
||||
{
|
||||
func(toStringRef(v.getFirst()), v.getSecond());
|
||||
}
|
||||
|
||||
for (auto & v : this->m3)
|
||||
{
|
||||
func(toStringRef(v.getFirst()), v.getSecond());
|
||||
}
|
||||
|
||||
for (auto & v : this->ms)
|
||||
{
|
||||
func(v.getFirst(), v.getSecond());
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Func>
|
||||
void ALWAYS_INLINE forEachMapped(Func && func)
|
||||
{
|
||||
if (this->m0.size())
|
||||
func(this->m0.zeroValue()->getSecond());
|
||||
for (auto & v : this->m1)
|
||||
func(v.getSecond());
|
||||
for (auto & v : this->m2)
|
||||
func(v.getSecond());
|
||||
for (auto & v : this->m3)
|
||||
func(v.getSecond());
|
||||
for (auto & v : this->ms)
|
||||
func(v.getSecond());
|
||||
}
|
||||
};
|
371
dbms/src/Common/HashTable/StringHashTable.h
Normal file
371
dbms/src/Common/HashTable/StringHashTable.h
Normal file
@ -0,0 +1,371 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashTable.h>
|
||||
|
||||
struct StringKey0
|
||||
{
|
||||
};
|
||||
|
||||
using StringKey8 = UInt64;
|
||||
using StringKey16 = DB::UInt128;
|
||||
struct StringKey24
|
||||
{
|
||||
UInt64 a;
|
||||
UInt64 b;
|
||||
UInt64 c;
|
||||
|
||||
bool operator==(const StringKey24 rhs) const { return a == rhs.a && b == rhs.b && c == rhs.c; }
|
||||
};
|
||||
|
||||
inline StringRef ALWAYS_INLINE toStringRef(const StringKey8 & n)
|
||||
{
|
||||
return {reinterpret_cast<const char *>(&n), 8ul - (__builtin_clzll(n) >> 3)};
|
||||
}
|
||||
inline StringRef ALWAYS_INLINE toStringRef(const StringKey16 & n)
|
||||
{
|
||||
return {reinterpret_cast<const char *>(&n), 16ul - (__builtin_clzll(n.high) >> 3)};
|
||||
}
|
||||
inline StringRef ALWAYS_INLINE toStringRef(const StringKey24 & n)
|
||||
{
|
||||
return {reinterpret_cast<const char *>(&n), 24ul - (__builtin_clzll(n.c) >> 3)};
|
||||
}
|
||||
|
||||
struct StringHashTableHash
|
||||
{
|
||||
#if defined(__SSE4_2__)
|
||||
size_t ALWAYS_INLINE operator()(StringKey8 key) const
|
||||
{
|
||||
size_t res = -1ULL;
|
||||
res = _mm_crc32_u64(res, key);
|
||||
return res;
|
||||
}
|
||||
size_t ALWAYS_INLINE operator()(StringKey16 key) const
|
||||
{
|
||||
size_t res = -1ULL;
|
||||
res = _mm_crc32_u64(res, key.low);
|
||||
res = _mm_crc32_u64(res, key.high);
|
||||
return res;
|
||||
}
|
||||
size_t ALWAYS_INLINE operator()(StringKey24 key) const
|
||||
{
|
||||
size_t res = -1ULL;
|
||||
res = _mm_crc32_u64(res, key.a);
|
||||
res = _mm_crc32_u64(res, key.b);
|
||||
res = _mm_crc32_u64(res, key.c);
|
||||
return res;
|
||||
}
|
||||
#else
|
||||
size_t ALWAYS_INLINE operator()(StringKey8 key) const
|
||||
{
|
||||
return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 8);
|
||||
}
|
||||
size_t ALWAYS_INLINE operator()(StringKey16 key) const
|
||||
{
|
||||
return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 16);
|
||||
}
|
||||
size_t ALWAYS_INLINE operator()(StringKey24 key) const
|
||||
{
|
||||
return CityHash_v1_0_2::CityHash64(reinterpret_cast<const char *>(&key), 24);
|
||||
}
|
||||
#endif
|
||||
size_t ALWAYS_INLINE operator()(StringRef key) const
|
||||
{
|
||||
return StringRefHash()(key);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename Cell>
|
||||
struct StringHashTableEmpty
|
||||
{
|
||||
using Self = StringHashTableEmpty;
|
||||
|
||||
bool has_zero = false;
|
||||
std::aligned_storage_t<sizeof(Cell), alignof(Cell)> zero_value_storage; /// Storage of element with zero key.
|
||||
|
||||
public:
|
||||
bool hasZero() const { return has_zero; }
|
||||
|
||||
void setHasZero()
|
||||
{
|
||||
has_zero = true;
|
||||
new (zeroValue()) Cell();
|
||||
}
|
||||
|
||||
void setHasZero(const Cell & other)
|
||||
{
|
||||
has_zero = true;
|
||||
new (zeroValue()) Cell(other);
|
||||
}
|
||||
|
||||
void clearHasZero()
|
||||
{
|
||||
has_zero = false;
|
||||
if (!std::is_trivially_destructible_v<Cell>)
|
||||
zeroValue()->~Cell();
|
||||
}
|
||||
|
||||
Cell * zeroValue() { return reinterpret_cast<Cell *>(&zero_value_storage); }
|
||||
const Cell * zeroValue() const { return reinterpret_cast<const Cell *>(&zero_value_storage); }
|
||||
|
||||
using LookupResult = Cell *;
|
||||
using ConstLookupResult = const Cell *;
|
||||
|
||||
template <typename KeyHolder>
|
||||
void ALWAYS_INLINE emplace(KeyHolder &&, LookupResult & it, bool & inserted, size_t /* hash */)
|
||||
{
|
||||
if (!hasZero())
|
||||
{
|
||||
setHasZero();
|
||||
inserted = true;
|
||||
}
|
||||
else
|
||||
inserted = false;
|
||||
it = zeroValue();
|
||||
}
|
||||
|
||||
template <typename Key>
|
||||
LookupResult ALWAYS_INLINE find(Key, size_t /* hash */)
|
||||
{
|
||||
return hasZero() ? zeroValue() : nullptr;
|
||||
}
|
||||
|
||||
|
||||
void write(DB::WriteBuffer & wb) const { zeroValue()->write(wb); }
|
||||
void writeText(DB::WriteBuffer & wb) const { zeroValue()->writeText(wb); }
|
||||
void read(DB::ReadBuffer & rb) { zeroValue()->read(rb); }
|
||||
void readText(DB::ReadBuffer & rb) { zeroValue()->readText(rb); }
|
||||
size_t size() const { return hasZero() ? 1 : 0; }
|
||||
bool empty() const { return !hasZero(); }
|
||||
size_t getBufferSizeInBytes() const { return sizeof(Cell); }
|
||||
size_t getCollisions() const { return 0; }
|
||||
};
|
||||
|
||||
template <size_t initial_size_degree = 8>
|
||||
struct StringHashTableGrower : public HashTableGrower<initial_size_degree>
|
||||
{
|
||||
// Smooth growing for string maps
|
||||
void increaseSize() { this->size_degree += 1; }
|
||||
};
|
||||
|
||||
template <typename SubMaps>
|
||||
class StringHashTable : private boost::noncopyable
|
||||
{
|
||||
protected:
|
||||
static constexpr size_t NUM_MAPS = 5;
|
||||
// Map for storing empty string
|
||||
using T0 = typename SubMaps::T0;
|
||||
|
||||
// Short strings are stored as numbers
|
||||
using T1 = typename SubMaps::T1;
|
||||
using T2 = typename SubMaps::T2;
|
||||
using T3 = typename SubMaps::T3;
|
||||
|
||||
// Long strings are stored as StringRef along with saved hash
|
||||
using Ts = typename SubMaps::Ts;
|
||||
using Self = StringHashTable;
|
||||
|
||||
template <typename, typename, size_t>
|
||||
friend class TwoLevelStringHashTable;
|
||||
|
||||
T0 m0;
|
||||
T1 m1;
|
||||
T2 m2;
|
||||
T3 m3;
|
||||
Ts ms;
|
||||
|
||||
public:
|
||||
using Key = StringRef;
|
||||
using key_type = Key;
|
||||
using value_type = typename Ts::value_type;
|
||||
using LookupResult = typename Ts::mapped_type *;
|
||||
|
||||
StringHashTable() {}
|
||||
|
||||
StringHashTable(size_t reserve_for_num_elements)
|
||||
: m1{reserve_for_num_elements / 4}
|
||||
, m2{reserve_for_num_elements / 4}
|
||||
, m3{reserve_for_num_elements / 4}
|
||||
, ms{reserve_for_num_elements / 4}
|
||||
{
|
||||
}
|
||||
|
||||
StringHashTable(StringHashTable && rhs) { *this = std::move(rhs); }
|
||||
~StringHashTable() {}
|
||||
|
||||
public:
|
||||
// Dispatch is written in a way that maximizes the performance:
|
||||
// 1. Always memcpy 8 times bytes
|
||||
// 2. Use switch case extension to generate fast dispatching table
|
||||
// 3. Funcs are named callables that can be force_inlined
|
||||
// NOTE: It relies on Little Endianness
|
||||
template <typename KeyHolder, typename Func>
|
||||
decltype(auto) ALWAYS_INLINE dispatch(KeyHolder && key_holder, Func && func)
|
||||
{
|
||||
const StringRef & x = keyHolderGetKey(key_holder);
|
||||
const size_t sz = x.size;
|
||||
if (sz == 0)
|
||||
{
|
||||
static constexpr StringKey0 key0{};
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(m0, key0, 0);
|
||||
}
|
||||
|
||||
const char * p = x.data;
|
||||
// pending bits that needs to be shifted out
|
||||
const char s = (-sz & 7) * 8;
|
||||
union
|
||||
{
|
||||
StringKey8 k8;
|
||||
StringKey16 k16;
|
||||
StringKey24 k24;
|
||||
UInt64 n[3];
|
||||
};
|
||||
StringHashTableHash hash;
|
||||
switch ((sz - 1) >> 3)
|
||||
{
|
||||
case 0: // 1..8 bytes
|
||||
{
|
||||
// first half page
|
||||
if ((reinterpret_cast<uintptr_t>(p) & 2048) == 0)
|
||||
{
|
||||
memcpy(&n[0], p, 8);
|
||||
n[0] &= -1ul >> s;
|
||||
}
|
||||
else
|
||||
{
|
||||
const char * lp = x.data + x.size - 8;
|
||||
memcpy(&n[0], lp, 8);
|
||||
n[0] >>= s;
|
||||
}
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(m1, k8, hash(k8));
|
||||
}
|
||||
case 1: // 9..16 bytes
|
||||
{
|
||||
memcpy(&n[0], p, 8);
|
||||
const char * lp = x.data + x.size - 8;
|
||||
memcpy(&n[1], lp, 8);
|
||||
n[1] >>= s;
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(m2, k16, hash(k16));
|
||||
}
|
||||
case 2: // 17..24 bytes
|
||||
{
|
||||
memcpy(&n[0], p, 16);
|
||||
const char * lp = x.data + x.size - 8;
|
||||
memcpy(&n[2], lp, 8);
|
||||
n[2] >>= s;
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(m3, k24, hash(k24));
|
||||
}
|
||||
default: // >= 25 bytes
|
||||
{
|
||||
return func(ms, std::forward<KeyHolder>(key_holder), hash(x));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct EmplaceCallable
|
||||
{
|
||||
LookupResult & mapped;
|
||||
bool & inserted;
|
||||
|
||||
EmplaceCallable(LookupResult & mapped_, bool & inserted_)
|
||||
: mapped(mapped_), inserted(inserted_) {}
|
||||
|
||||
template <typename Map, typename KeyHolder>
|
||||
void ALWAYS_INLINE operator()(Map & map, KeyHolder && key_holder, size_t hash)
|
||||
{
|
||||
typename Map::LookupResult result;
|
||||
map.emplace(key_holder, result, inserted, hash);
|
||||
mapped = lookupResultGetMapped(result);
|
||||
}
|
||||
};
|
||||
|
||||
template <typename KeyHolder>
|
||||
void ALWAYS_INLINE emplace(KeyHolder && key_holder, LookupResult & it, bool & inserted)
|
||||
{
|
||||
this->dispatch(key_holder, EmplaceCallable(it, inserted));
|
||||
}
|
||||
|
||||
struct FindCallable
|
||||
{
|
||||
// find() doesn't need any key memory management, so we don't work with
|
||||
// any key holders here, only with normal keys. The key type is still
|
||||
// different for every subtable, this is why it is a template parameter.
|
||||
template <typename Submap, typename SubmapKey>
|
||||
LookupResult ALWAYS_INLINE operator()(Submap & map, const SubmapKey & key, size_t hash)
|
||||
{
|
||||
return lookupResultGetMapped(map.find(key, hash));
|
||||
}
|
||||
};
|
||||
|
||||
LookupResult ALWAYS_INLINE find(Key x)
|
||||
{
|
||||
return dispatch(x, FindCallable{});
|
||||
}
|
||||
|
||||
void write(DB::WriteBuffer & wb) const
|
||||
{
|
||||
m0.write(wb);
|
||||
m1.write(wb);
|
||||
m2.write(wb);
|
||||
m3.write(wb);
|
||||
ms.write(wb);
|
||||
}
|
||||
|
||||
void writeText(DB::WriteBuffer & wb) const
|
||||
{
|
||||
m0.writeText(wb);
|
||||
DB::writeChar(',', wb);
|
||||
m1.writeText(wb);
|
||||
DB::writeChar(',', wb);
|
||||
m2.writeText(wb);
|
||||
DB::writeChar(',', wb);
|
||||
m3.writeText(wb);
|
||||
DB::writeChar(',', wb);
|
||||
ms.writeText(wb);
|
||||
}
|
||||
|
||||
void read(DB::ReadBuffer & rb)
|
||||
{
|
||||
m0.read(rb);
|
||||
m1.read(rb);
|
||||
m2.read(rb);
|
||||
m3.read(rb);
|
||||
ms.read(rb);
|
||||
}
|
||||
|
||||
void readText(DB::ReadBuffer & rb)
|
||||
{
|
||||
m0.readText(rb);
|
||||
DB::assertChar(',', rb);
|
||||
m1.readText(rb);
|
||||
DB::assertChar(',', rb);
|
||||
m2.readText(rb);
|
||||
DB::assertChar(',', rb);
|
||||
m3.readText(rb);
|
||||
DB::assertChar(',', rb);
|
||||
ms.readText(rb);
|
||||
}
|
||||
|
||||
size_t size() const { return m0.size() + m1.size() + m2.size() + m3.size() + ms.size(); }
|
||||
|
||||
bool empty() const { return m0.empty() && m1.empty() && m2.empty() && m3.empty() && ms.empty(); }
|
||||
|
||||
size_t getBufferSizeInBytes() const
|
||||
{
|
||||
return m0.getBufferSizeInBytes() + m1.getBufferSizeInBytes() + m2.getBufferSizeInBytes() + m3.getBufferSizeInBytes()
|
||||
+ ms.getBufferSizeInBytes();
|
||||
}
|
||||
|
||||
void clearAndShrink()
|
||||
{
|
||||
m1.clearHasZero();
|
||||
m1.clearAndShrink();
|
||||
m2.clearAndShrink();
|
||||
m3.clearAndShrink();
|
||||
ms.clearAndShrink();
|
||||
}
|
||||
};
|
37
dbms/src/Common/HashTable/TwoLevelStringHashMap.h
Normal file
37
dbms/src/Common/HashTable/TwoLevelStringHashMap.h
Normal file
@ -0,0 +1,37 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/HashTable/StringHashMap.h>
|
||||
#include <Common/HashTable/TwoLevelStringHashTable.h>
|
||||
|
||||
template <typename TMapped, typename Allocator = HashTableAllocator, template <typename...> typename ImplTable = StringHashMap>
|
||||
class TwoLevelStringHashMap : public TwoLevelStringHashTable<StringHashMapSubMaps<TMapped, Allocator>, ImplTable<TMapped, Allocator>>
|
||||
{
|
||||
public:
|
||||
using Key = StringRef;
|
||||
using key_type = Key;
|
||||
using Self = TwoLevelStringHashMap;
|
||||
using Base = TwoLevelStringHashTable<StringHashMapSubMaps<TMapped, Allocator>, StringHashMap<TMapped, Allocator>>;
|
||||
using Base::Base;
|
||||
using typename Base::Impl;
|
||||
using mapped_type = TMapped;
|
||||
using value_type = typename Base::value_type;
|
||||
|
||||
using LookupResult = typename Base::LookupResult;
|
||||
|
||||
template <typename Func>
|
||||
void ALWAYS_INLINE forEachMapped(Func && func)
|
||||
{
|
||||
for (auto i = 0u; i < this->NUM_BUCKETS; ++i)
|
||||
return this->impls[i].forEachMapped(func);
|
||||
}
|
||||
|
||||
mapped_type & ALWAYS_INLINE operator[](Key x)
|
||||
{
|
||||
bool inserted;
|
||||
LookupResult it;
|
||||
emplace(x, it, inserted);
|
||||
if (inserted)
|
||||
new (lookupResultGetMapped(it)) mapped_type();
|
||||
return *lookupResultGetMapped(it);
|
||||
}
|
||||
};
|
218
dbms/src/Common/HashTable/TwoLevelStringHashTable.h
Normal file
218
dbms/src/Common/HashTable/TwoLevelStringHashTable.h
Normal file
@ -0,0 +1,218 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/HashTable/StringHashTable.h>
|
||||
|
||||
template <typename SubMaps, typename ImplTable = StringHashTable<SubMaps>, size_t BITS_FOR_BUCKET = 8>
|
||||
class TwoLevelStringHashTable : private boost::noncopyable
|
||||
{
|
||||
protected:
|
||||
using HashValue = size_t;
|
||||
using Self = TwoLevelStringHashTable;
|
||||
|
||||
public:
|
||||
using Key = StringRef;
|
||||
using Impl = ImplTable;
|
||||
|
||||
static constexpr size_t NUM_BUCKETS = 1ULL << BITS_FOR_BUCKET;
|
||||
static constexpr size_t MAX_BUCKET = NUM_BUCKETS - 1;
|
||||
|
||||
// TODO: currently hashing contains redundant computations when doing distributed or external aggregations
|
||||
size_t hash(const Key & x) const
|
||||
{
|
||||
return const_cast<Self &>(*this).dispatch(x,
|
||||
[&](const auto &, const auto &, size_t hash) { return hash; });
|
||||
}
|
||||
|
||||
size_t operator()(const Key & x) const { return hash(x); }
|
||||
|
||||
/// NOTE Bad for hash tables with more than 2^32 cells.
|
||||
static size_t getBucketFromHash(size_t hash_value) { return (hash_value >> (32 - BITS_FOR_BUCKET)) & MAX_BUCKET; }
|
||||
|
||||
public:
|
||||
using key_type = typename Impl::key_type;
|
||||
using value_type = typename Impl::value_type;
|
||||
using LookupResult = typename Impl::LookupResult;
|
||||
|
||||
Impl impls[NUM_BUCKETS];
|
||||
|
||||
TwoLevelStringHashTable() {}
|
||||
|
||||
template <typename Source>
|
||||
TwoLevelStringHashTable(const Source & src)
|
||||
{
|
||||
if (src.m0.hasZero())
|
||||
impls[0].m0.setHasZero(*src.m0.zeroValue());
|
||||
|
||||
for (auto & v : src.m1)
|
||||
{
|
||||
size_t hash_value = v.getHash(src.m1);
|
||||
size_t buck = getBucketFromHash(hash_value);
|
||||
impls[buck].m1.insertUniqueNonZero(&v, hash_value);
|
||||
}
|
||||
for (auto & v : src.m2)
|
||||
{
|
||||
size_t hash_value = v.getHash(src.m2);
|
||||
size_t buck = getBucketFromHash(hash_value);
|
||||
impls[buck].m2.insertUniqueNonZero(&v, hash_value);
|
||||
}
|
||||
for (auto & v : src.m3)
|
||||
{
|
||||
size_t hash_value = v.getHash(src.m3);
|
||||
size_t buck = getBucketFromHash(hash_value);
|
||||
impls[buck].m3.insertUniqueNonZero(&v, hash_value);
|
||||
}
|
||||
for (auto & v : src.ms)
|
||||
{
|
||||
size_t hash_value = v.getHash(src.ms);
|
||||
size_t buck = getBucketFromHash(hash_value);
|
||||
impls[buck].ms.insertUniqueNonZero(&v, hash_value);
|
||||
}
|
||||
}
|
||||
|
||||
// This function is mostly the same as StringHashTable::dispatch, but with
|
||||
// added bucket computation. See the comments there.
|
||||
template <typename Func, typename KeyHolder>
|
||||
decltype(auto) ALWAYS_INLINE dispatch(KeyHolder && key_holder, Func && func)
|
||||
{
|
||||
const StringRef & x = keyHolderGetKey(key_holder);
|
||||
const size_t sz = x.size;
|
||||
if (sz == 0)
|
||||
{
|
||||
static constexpr StringKey0 key0{};
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(impls[0].m0, key0, 0);
|
||||
}
|
||||
|
||||
const char * p = x.data;
|
||||
// pending bits that needs to be shifted out
|
||||
const char s = (-sz & 7) * 8;
|
||||
union
|
||||
{
|
||||
StringKey8 k8;
|
||||
StringKey16 k16;
|
||||
StringKey24 k24;
|
||||
UInt64 n[3];
|
||||
};
|
||||
StringHashTableHash hash;
|
||||
switch ((sz - 1) >> 3)
|
||||
{
|
||||
case 0:
|
||||
{
|
||||
// first half page
|
||||
if ((reinterpret_cast<uintptr_t>(p) & 2048) == 0)
|
||||
{
|
||||
memcpy(&n[0], p, 8);
|
||||
n[0] &= -1ul >> s;
|
||||
}
|
||||
else
|
||||
{
|
||||
const char * lp = x.data + x.size - 8;
|
||||
memcpy(&n[0], lp, 8);
|
||||
n[0] >>= s;
|
||||
}
|
||||
auto res = hash(k8);
|
||||
auto buck = getBucketFromHash(res);
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(impls[buck].m1, k8, res);
|
||||
}
|
||||
case 1:
|
||||
{
|
||||
memcpy(&n[0], p, 8);
|
||||
const char * lp = x.data + x.size - 8;
|
||||
memcpy(&n[1], lp, 8);
|
||||
n[1] >>= s;
|
||||
auto res = hash(k16);
|
||||
auto buck = getBucketFromHash(res);
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(impls[buck].m2, k16, res);
|
||||
}
|
||||
case 2:
|
||||
{
|
||||
memcpy(&n[0], p, 16);
|
||||
const char * lp = x.data + x.size - 8;
|
||||
memcpy(&n[2], lp, 8);
|
||||
n[2] >>= s;
|
||||
auto res = hash(k24);
|
||||
auto buck = getBucketFromHash(res);
|
||||
keyHolderDiscardKey(key_holder);
|
||||
return func(impls[buck].m3, k24, res);
|
||||
}
|
||||
default:
|
||||
{
|
||||
auto res = hash(x);
|
||||
auto buck = getBucketFromHash(res);
|
||||
return func(impls[buck].ms, std::forward<KeyHolder>(key_holder), res);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <typename KeyHolder>
|
||||
void ALWAYS_INLINE emplace(KeyHolder && key_holder, LookupResult & it, bool & inserted)
|
||||
{
|
||||
dispatch(key_holder, typename Impl::EmplaceCallable{it, inserted});
|
||||
}
|
||||
|
||||
LookupResult ALWAYS_INLINE find(Key x)
|
||||
{
|
||||
return dispatch(x, typename Impl::FindCallable{});
|
||||
}
|
||||
|
||||
void write(DB::WriteBuffer & wb) const
|
||||
{
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
impls[i].write(wb);
|
||||
}
|
||||
|
||||
void writeText(DB::WriteBuffer & wb) const
|
||||
{
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
DB::writeChar(',', wb);
|
||||
impls[i].writeText(wb);
|
||||
}
|
||||
}
|
||||
|
||||
void read(DB::ReadBuffer & rb)
|
||||
{
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
impls[i].read(rb);
|
||||
}
|
||||
|
||||
void readText(DB::ReadBuffer & rb)
|
||||
{
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
DB::assertChar(',', rb);
|
||||
impls[i].readText(rb);
|
||||
}
|
||||
}
|
||||
|
||||
size_t size() const
|
||||
{
|
||||
size_t res = 0;
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
res += impls[i].size();
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
if (!impls[i].empty())
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t getBufferSizeInBytes() const
|
||||
{
|
||||
size_t res = 0;
|
||||
for (size_t i = 0; i < NUM_BUCKETS; ++i)
|
||||
res += impls[i].getBufferSizeInBytes();
|
||||
|
||||
return res;
|
||||
}
|
||||
};
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -380,6 +380,8 @@ struct Settings : public SettingsCollection<Settings>
|
||||
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(SettingDefaultDatabaseEngine, default_database_engine, /*DefaultDatabaseEngine::Ordinary*/ DefaultDatabaseEngine::Atomic, "Default database engine.") \
|
||||
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") \
|
||||
|
@ -6,11 +6,18 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// Used to disable implicit casting for certain overloaded types such as Field, which leads to
|
||||
// overload resolution ambiguity.
|
||||
template <typename T> struct Dumpable;
|
||||
template <typename T>
|
||||
std::ostream & operator<<(std::ostream & stream, const typename Dumpable<T>::Type & what);
|
||||
|
||||
class IBlockInputStream;
|
||||
std::ostream & operator<<(std::ostream & stream, const IBlockInputStream & what);
|
||||
|
||||
class Field;
|
||||
std::ostream & operator<<(std::ostream & stream, const Field & what);
|
||||
template <> struct Dumpable<Field> { using Type = Field; };
|
||||
|
||||
struct NameAndTypePair;
|
||||
std::ostream & operator<<(std::ostream & stream, const NameAndTypePair & what);
|
||||
|
@ -23,8 +23,8 @@ namespace ErrorCodes
|
||||
RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
Connection & connection,
|
||||
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
|
||||
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
if (settings)
|
||||
context.setSettings(*settings);
|
||||
@ -38,8 +38,8 @@ RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
|
||||
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
if (settings)
|
||||
context.setSettings(*settings);
|
||||
@ -54,8 +54,8 @@ RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
RemoteBlockInputStream::RemoteBlockInputStream(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_, const Block & header_, const Context & context_, const Settings * settings,
|
||||
const ThrottlerPtr & throttler, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), external_tables(external_tables_), stage(stage_)
|
||||
const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_)
|
||||
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
|
||||
{
|
||||
if (settings)
|
||||
context.setSettings(*settings);
|
||||
@ -120,6 +120,11 @@ void RemoteBlockInputStream::cancel(bool kill)
|
||||
tryCancel("Cancelling query");
|
||||
}
|
||||
|
||||
void RemoteBlockInputStream::sendScalars()
|
||||
{
|
||||
multiplexed_connections->sendScalarsData(scalars);
|
||||
}
|
||||
|
||||
void RemoteBlockInputStream::sendExternalTables()
|
||||
{
|
||||
size_t count = multiplexed_connections->size();
|
||||
@ -308,6 +313,8 @@ void RemoteBlockInputStream::sendQuery()
|
||||
established = false;
|
||||
sent_query = true;
|
||||
|
||||
if (settings.enable_scalar_subquery_optimization)
|
||||
sendScalars();
|
||||
sendExternalTables();
|
||||
}
|
||||
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
RemoteBlockInputStream(
|
||||
Connection & connection,
|
||||
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
|
||||
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
/// Accepts several connections already taken from pool.
|
||||
@ -33,7 +33,7 @@ public:
|
||||
RemoteBlockInputStream(
|
||||
std::vector<IConnectionPool::Entry> && connections,
|
||||
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
|
||||
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
/// Takes a pool and gets one or several connections from it.
|
||||
@ -41,7 +41,7 @@ public:
|
||||
RemoteBlockInputStream(
|
||||
const ConnectionPoolWithFailoverPtr & pool,
|
||||
const String & query_, const Block & header_, const Context & context_, const Settings * settings = nullptr,
|
||||
const ThrottlerPtr & throttler = nullptr, const Tables & external_tables_ = Tables(),
|
||||
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
|
||||
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete);
|
||||
|
||||
~RemoteBlockInputStream() override;
|
||||
@ -71,6 +71,9 @@ public:
|
||||
Block getHeader() const override { return header; }
|
||||
|
||||
protected:
|
||||
/// Send all scalars to remote servers
|
||||
void sendScalars();
|
||||
|
||||
/// Send all temporary tables to remote servers
|
||||
void sendExternalTables();
|
||||
|
||||
@ -103,6 +106,8 @@ private:
|
||||
String query_id = "";
|
||||
Context context;
|
||||
|
||||
/// Scalars needed to be sent to remote servers
|
||||
Scalars scalars;
|
||||
/// Temporary tables needed to be sent to remote servers
|
||||
Tables external_tables;
|
||||
QueryProcessingStage::Enum stage;
|
||||
|
@ -19,6 +19,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_)
|
||||
@ -27,32 +28,36 @@ DatabaseDictionary::DatabaseDictionary(const String & name_)
|
||||
{
|
||||
}
|
||||
|
||||
void DatabaseDictionary::loadTables(Context &, bool)
|
||||
void DatabaseDictionary::loadStoredObjects(Context &, bool)
|
||||
{
|
||||
}
|
||||
|
||||
Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
Tables tables;
|
||||
ExternalLoader::Loadables loadables;
|
||||
ExternalLoader::LoadResults load_results;
|
||||
if (filter_by_name)
|
||||
{
|
||||
/// If `filter_by_name` is set, we iterate through all dictionaries with such names. That's why we need to load all of them.
|
||||
loadables = context.getExternalDictionariesLoader().loadAndGet(filter_by_name);
|
||||
context.getExternalDictionariesLoader().load(filter_by_name, load_results);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If `filter_by_name` isn't set, we iterate through only already loaded dictionaries. We don't try to load all dictionaries in this case.
|
||||
loadables = context.getExternalDictionariesLoader().getCurrentlyLoadedObjects();
|
||||
load_results = context.getExternalDictionariesLoader().getCurrentLoadResults();
|
||||
}
|
||||
|
||||
for (const auto & loadable : loadables)
|
||||
for (const auto & [object_name, info]: load_results)
|
||||
{
|
||||
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(loadable);
|
||||
auto dict_name = dict_ptr->getName();
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
/// Load tables only from XML dictionaries, don't touch other
|
||||
if (info.object != nullptr && info.repository_name.empty())
|
||||
{
|
||||
auto dict_ptr = std::static_pointer_cast<const IDictionaryBase>(info.object);
|
||||
auto dict_name = dict_ptr->getName();
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
tables[dict_name] = StorageDictionary::create(getDatabaseName(), dict_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
}
|
||||
}
|
||||
return tables;
|
||||
}
|
||||
@ -64,6 +69,66 @@ bool DatabaseDictionary::isTableExist(
|
||||
return context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
|
||||
}
|
||||
|
||||
|
||||
bool DatabaseDictionary::isDictionaryExist(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
DatabaseDictionariesIteratorPtr DatabaseDictionary::getDictionariesIterator(
|
||||
const Context & /*context*/,
|
||||
const FilterByNameFunction & /*filter_by_dictionary_name*/)
|
||||
{
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseDictionary::createDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*dictionary_name*/,
|
||||
const ASTPtr & /*query*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseDictionary::removeDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseDictionary::attachDictionary(
|
||||
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseDictionary::detachDictionary(
|
||||
const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseDictionary::tryGetCreateDictionaryQuery(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateDictionaryQuery(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
throw Exception("Dictionary engine doesn't support dictionaries.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
StoragePtr DatabaseDictionary::tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
@ -79,9 +144,9 @@ StoragePtr DatabaseDictionary::tryGetTable(
|
||||
return {};
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
return std::make_unique<DatabaseSnapshotIterator>(listTables(context, filter_by_name));
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(context, filter_by_name));
|
||||
}
|
||||
|
||||
bool DatabaseDictionary::empty(const Context & context) const
|
||||
@ -115,7 +180,7 @@ void DatabaseDictionary::removeTable(
|
||||
throw Exception("DatabaseDictionary: removeTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
time_t DatabaseDictionary::getTableMetadataModificationTime(
|
||||
time_t DatabaseDictionary::getObjectMetadataModificationTime(
|
||||
const Context &,
|
||||
const String &)
|
||||
{
|
||||
|
@ -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:
|
||||
|
@ -56,11 +56,13 @@ DatabasePtr DatabaseFactory::get(
|
||||
else if (engine_name == "MySQL")
|
||||
{
|
||||
const ASTFunction * engine = engine_define->engine;
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
if (arguments.size() != 4)
|
||||
throw Exception("MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!engine->arguments || engine->arguments->children.size() != 4)
|
||||
throw Exception(
|
||||
"MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
const auto & mysql_host_name = arguments[0]->as<ASTLiteral>()->value.safeGet<String>();
|
||||
const auto & mysql_database_name = arguments[1]->as<ASTLiteral>()->value.safeGet<String>();
|
||||
@ -77,11 +79,11 @@ DatabasePtr DatabaseFactory::get(
|
||||
else if (engine_name == "Lazy")
|
||||
{
|
||||
const ASTFunction * engine = engine_define->engine;
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Lazy database require cache_expiration_time_seconds argument.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!engine->arguments || engine->arguments->children.size() != 1)
|
||||
throw Exception("Lazy database require cache_expiration_time_seconds argument", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
const auto & arguments = engine->arguments->children;
|
||||
|
||||
const auto cache_expiration_time_seconds = arguments[0]->as<ASTLiteral>()->value.safeGet<UInt64>();
|
||||
return std::make_shared<DatabaseLazy>(database_name, metadata_path, cache_expiration_time_seconds, context);
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
@ -29,8 +28,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 +40,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, context, [this](const String & file_name)
|
||||
DatabaseOnDisk::iterateMetadataFiles(*this, log, context, [this](const String & file_name)
|
||||
{
|
||||
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
||||
attachTable(table_name, nullptr);
|
||||
@ -70,7 +67,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 +88,51 @@ void DatabaseLazy::removeTable(
|
||||
DatabaseOnDisk::removeTable(*this, context, table_name, log);
|
||||
}
|
||||
|
||||
void DatabaseLazy::removeDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseLazy::getCreateDictionaryQuery(
|
||||
const Context & /*context*/,
|
||||
const String & /*table_name*/) const
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseLazy::tryGetCreateDictionaryQuery(const Context & /*context*/, const String & /*table_name*/) const
|
||||
{
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
bool DatabaseLazy::isDictionaryExist(const Context & /*context*/, const String & /*table_name*/) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
DatabaseDictionariesIteratorPtr DatabaseLazy::getDictionariesIterator(
|
||||
const Context & /*context*/,
|
||||
const FilterByNameFunction & /*filter_by_dictionary_name*/)
|
||||
{
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
|
||||
}
|
||||
|
||||
void DatabaseLazy::attachDictionary(
|
||||
const String & /*dictionary_name*/,
|
||||
const Context & /*context*/,
|
||||
bool /*load*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseLazy::detachDictionary(const String & /*dictionary_name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseLazy::renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -94,7 +145,7 @@ void DatabaseLazy::renameTable(
|
||||
}
|
||||
|
||||
|
||||
time_t DatabaseLazy::getTableMetadataModificationTime(
|
||||
time_t DatabaseLazy::getObjectMetadataModificationTime(
|
||||
const Context & /* context */,
|
||||
const String & table_name)
|
||||
{
|
||||
@ -172,7 +223,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 +249,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 +318,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 +331,18 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
|
||||
|
||||
const String table_metadata_path = getMetadataPath() + "/" + escapeForFileName(table_name) + ".sql";
|
||||
|
||||
String s;
|
||||
{
|
||||
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
||||
ReadBufferFromFile in(table_metadata_path, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
||||
readStringUntilEOF(s, in);
|
||||
}
|
||||
|
||||
/** Empty files with metadata are generated after a rough restart of the server.
|
||||
* Remove these files to slightly reduce the work of the admins on startup.
|
||||
*/
|
||||
if (s.empty())
|
||||
{
|
||||
LOG_ERROR(log, "LoadTable: File " << table_metadata_path << " is empty. Removing.");
|
||||
Poco::File(table_metadata_path).remove();
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
String table_name_;
|
||||
StoragePtr table;
|
||||
Context context_copy(context); /// some tables can change context, but not LogTables
|
||||
std::tie(table_name_, table) = createTableFromDefinition(
|
||||
s, name, getDataPath(), context_copy, false, "in file " + table_metadata_path);
|
||||
if (!endsWith(table->getName(), "Log"))
|
||||
|
||||
auto ast = parseCreateQueryFromMetadataFile(table_metadata_path, log);
|
||||
if (ast)
|
||||
std::tie(table_name_, table) = createTableFromAST(
|
||||
ast->as<const ASTCreateQuery &>(), name, getDataPath(), context_copy, false);
|
||||
|
||||
if (!ast || !endsWith(table->getName(), "Log"))
|
||||
throw Exception("Only *Log tables can be used with Lazy database engine.", ErrorCodes::LOGICAL_ERROR);
|
||||
{
|
||||
std::lock_guard lock(tables_mutex);
|
||||
|
@ -2,11 +2,13 @@
|
||||
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class DatabaseLazyIterator;
|
||||
|
||||
/** Lazy engine of databases.
|
||||
@ -20,7 +22,7 @@ public:
|
||||
|
||||
String getEngineName() const override { return "Lazy"; }
|
||||
|
||||
void loadTables(
|
||||
void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag) override;
|
||||
|
||||
@ -30,10 +32,19 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void createDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void removeTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void removeDictionary(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -49,7 +60,7 @@ public:
|
||||
const ConstraintsDescription & constraints,
|
||||
const ASTModifier & engine_modifier) override;
|
||||
|
||||
time_t getTableMetadataModificationTime(
|
||||
time_t getObjectMetadataModificationTime(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
@ -61,12 +72,20 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr getCreateDictionaryQuery(
|
||||
const Context & context,
|
||||
const String & dictionary_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateDictionaryQuery(
|
||||
const Context & context,
|
||||
const String & dictionary_name) const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
String getDataPath() const override;
|
||||
String getDatabaseName() const override;
|
||||
String getMetadataPath() const override;
|
||||
String getTableMetadataPath(const String & table_name) const override;
|
||||
String getObjectMetadataPath(const String & table_name) const override;
|
||||
|
||||
void drop(const Context & context) 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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,6 +9,8 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
DatabaseMemory::DatabaseMemory(String name_)
|
||||
@ -16,7 +18,7 @@ DatabaseMemory::DatabaseMemory(String name_)
|
||||
, log(&Logger::get("DatabaseMemory(" + name + ")"))
|
||||
{}
|
||||
|
||||
void DatabaseMemory::loadTables(
|
||||
void DatabaseMemory::loadStoredObjects(
|
||||
Context & /*context*/,
|
||||
bool /*has_force_restore_data_flag*/)
|
||||
{
|
||||
@ -32,6 +34,21 @@ void DatabaseMemory::createTable(
|
||||
attachTable(table_name, table);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::attachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("There is no ATTACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
void DatabaseMemory::createDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*dictionary_name*/,
|
||||
const ASTPtr & /*query*/)
|
||||
{
|
||||
throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::removeTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name)
|
||||
@ -39,9 +56,23 @@ void DatabaseMemory::removeTable(
|
||||
detachTable(table_name);
|
||||
}
|
||||
|
||||
time_t DatabaseMemory::getTableMetadataModificationTime(
|
||||
const Context &,
|
||||
const String &)
|
||||
|
||||
void DatabaseMemory::detachDictionary(const String & /*name*/, const Context & /*context*/, bool /*reload*/)
|
||||
{
|
||||
throw Exception("There is no DETACH DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
void DatabaseMemory::removeDictionary(
|
||||
const Context & /*context*/,
|
||||
const String & /*dictionary_name*/)
|
||||
{
|
||||
throw Exception("There is no DROP DICTIONARY query for DatabaseMemory", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
|
||||
time_t DatabaseMemory::getObjectMetadataModificationTime(
|
||||
const Context &, const String &)
|
||||
{
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
@ -53,6 +84,15 @@ ASTPtr DatabaseMemory::getCreateTableQuery(
|
||||
throw Exception("There is no CREATE TABLE query for DatabaseMemory tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateDictionaryQuery(
|
||||
const Context &,
|
||||
const String &) const
|
||||
{
|
||||
throw Exception("There is no CREATE DICTIONARY query for DatabaseMemory dictionaries", ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateDatabaseQuery(
|
||||
const Context &) const
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -64,7 +64,7 @@ bool DatabaseMySQL::empty(const Context &) const
|
||||
return local_tables_cache.empty();
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseMySQL::getIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
Tables tables;
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
@ -75,7 +75,7 @@ DatabaseIteratorPtr DatabaseMySQL::getIterator(const Context &, const FilterByNa
|
||||
if (!filter_by_table_name || filter_by_table_name(local_table.first))
|
||||
tables[local_table.first] = local_table.second.storage;
|
||||
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseMySQL::isTableExist(const Context & context, const String & name) const
|
||||
@ -107,7 +107,7 @@ ASTPtr DatabaseMySQL::tryGetCreateTableQuery(const Context &, const String & tab
|
||||
return local_tables_cache[table_name].create_table_query;
|
||||
}
|
||||
|
||||
time_t DatabaseMySQL::getTableMetadataModificationTime(const Context &, const String & table_name)
|
||||
time_t DatabaseMySQL::getObjectMetadataModificationTime(const Context &, const String & table_name)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <mysqlxx/Pool.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -29,17 +30,32 @@ public:
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) override;
|
||||
|
||||
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context &, const FilterByNameFunction & = {}) override
|
||||
{
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
|
||||
}
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
bool isTableExist(const Context & context, const String & name) const override;
|
||||
|
||||
bool isDictionaryExist(const Context &, const String &) const override { return false; }
|
||||
|
||||
StoragePtr tryGetTable(const Context & context, const String & name) const override;
|
||||
|
||||
ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const override;
|
||||
|
||||
time_t getTableMetadataModificationTime(const Context & context, const String & name) override;
|
||||
ASTPtr getCreateDictionaryQuery(const Context &, const String &) const override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support dictionaries.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
ASTPtr tryGetCreateDictionaryQuery(const Context &, const String &) const override { return nullptr; }
|
||||
|
||||
|
||||
time_t getObjectMetadataModificationTime(const Context & context, const String & name) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
@ -48,7 +64,12 @@ public:
|
||||
throw Exception("MySQL database engine does not support detach table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void loadTables(Context &, bool) override
|
||||
void detachDictionary(const String &, const Context &, bool) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support detach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void loadStoredObjects(Context &, bool) override
|
||||
{
|
||||
/// do nothing
|
||||
}
|
||||
@ -58,16 +79,33 @@ public:
|
||||
throw Exception("MySQL database engine does not support remove table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void removeDictionary(const Context &, const String &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support remove dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
void attachTable(const String &, const StoragePtr &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support attach table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void attachDictionary(const String &, const Context &, bool) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support attach dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void createTable(const Context &, const String &, const StoragePtr &, const ASTPtr &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support create table.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void createDictionary(const Context &, const String &, const ASTPtr &) override
|
||||
{
|
||||
throw Exception("MySQL database engine does not support create dictionary.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
struct MySQLStorageInfo
|
||||
{
|
||||
|
@ -1,37 +1,48 @@
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int CANNOT_GET_CREATE_DICTIONARY_QUERY;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int DICTIONARY_ALREADY_EXISTS;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
}
|
||||
|
||||
|
||||
namespace detail
|
||||
{
|
||||
String getTableMetadataPath(const String & base_path, const String & table_name)
|
||||
String getObjectMetadataPath(const String & base_path, const String & table_name)
|
||||
{
|
||||
return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql";
|
||||
}
|
||||
@ -85,6 +96,107 @@ namespace detail
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log)
|
||||
{
|
||||
String definition;
|
||||
{
|
||||
char in_buf[METADATA_FILE_BUFFER_SIZE];
|
||||
ReadBufferFromFile in(filepath, METADATA_FILE_BUFFER_SIZE, -1, in_buf);
|
||||
readStringUntilEOF(definition, in);
|
||||
}
|
||||
|
||||
/** Empty files with metadata are generated after a rough restart of the server.
|
||||
* Remove these files to slightly reduce the work of the admins on startup.
|
||||
*/
|
||||
if (definition.empty())
|
||||
{
|
||||
LOG_ERROR(log, "File " << filepath << " is empty. Removing.");
|
||||
Poco::File(filepath).remove();
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
ParserCreateQuery parser_create;
|
||||
ASTPtr result = parseQuery(parser_create, definition, "in file " + filepath, 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
|
||||
std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & database_data_path,
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.database = database_name;
|
||||
|
||||
if (ast_create_query.as_table_function)
|
||||
{
|
||||
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
||||
return {ast_create_query.table, storage};
|
||||
}
|
||||
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
||||
/// - the database has not been created yet;
|
||||
/// - the code is simpler, since the query is already brought to a suitable form.
|
||||
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
|
||||
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
|
||||
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context);
|
||||
ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
||||
|
||||
return
|
||||
{
|
||||
ast_create_query.table,
|
||||
StorageFactory::instance().get(
|
||||
ast_create_query,
|
||||
database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(),
|
||||
columns, constraints,
|
||||
true, has_force_restore_data_flag)
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
{
|
||||
ASTPtr query_clone = query->clone();
|
||||
auto * create = query_clone->as<ASTCreateQuery>();
|
||||
|
||||
if (!create)
|
||||
{
|
||||
std::ostringstream query_stream;
|
||||
formatAST(*create, query_stream, true);
|
||||
throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (!create->is_dictionary)
|
||||
create->attach = true;
|
||||
|
||||
/// We remove everything that is not needed for ATTACH from the query.
|
||||
create->database.clear();
|
||||
create->as_database.clear();
|
||||
create->as_table.clear();
|
||||
create->if_not_exists = false;
|
||||
create->is_populate = false;
|
||||
create->replace_view = false;
|
||||
|
||||
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
|
||||
if (!create->is_view && !create->is_materialized_view && !create->is_live_view)
|
||||
create->select = nullptr;
|
||||
|
||||
create->format = nullptr;
|
||||
create->out_file = nullptr;
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
formatAST(*create, statement_stream, false);
|
||||
statement_stream << '\n';
|
||||
return statement_stream.str();
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::createTable(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
@ -106,15 +218,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 +257,70 @@ void DatabaseOnDisk::createTable(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOnDisk::createDictionary(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
|
||||
/** The code is based on the assumption that all threads share the same order of operations
|
||||
* - creating the .sql.tmp file;
|
||||
* - adding a dictionary to `dictionaries`;
|
||||
* - rename .sql.tmp to .sql.
|
||||
*/
|
||||
|
||||
/// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH.
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
if (database.isDictionaryExist(context, dictionary_name))
|
||||
throw Exception("Dictionary " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
|
||||
if (database.isTableExist(context, dictionary_name))
|
||||
throw Exception("Table " + backQuote(database.getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
|
||||
String dictionary_metadata_path = database.getObjectMetadataPath(dictionary_name);
|
||||
String dictionary_metadata_tmp_path = dictionary_metadata_path + ".tmp";
|
||||
String statement;
|
||||
|
||||
{
|
||||
statement = getObjectDefinitionFromCreateQuery(query);
|
||||
|
||||
/// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown.
|
||||
WriteBufferFromFile out(dictionary_metadata_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
||||
writeString(statement, out);
|
||||
out.next();
|
||||
if (settings.fsync_metadata)
|
||||
out.sync();
|
||||
out.close();
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
/// Do not load it now because we want more strict loading
|
||||
database.attachDictionary(dictionary_name, context, false);
|
||||
/// Load dictionary
|
||||
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
|
||||
String dict_name = database.getDatabaseName() + "." + dictionary_name;
|
||||
context.getExternalDictionariesLoader().addDictionaryWithConfig(
|
||||
dict_name, database.getDatabaseName(), query->as<const ASTCreateQuery &>(), !lazy_load);
|
||||
|
||||
/// If it was ATTACH query and file with dictionary metadata already exist
|
||||
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
|
||||
Poco::File(dictionary_metadata_tmp_path).renameTo(dictionary_metadata_path);
|
||||
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
database.detachDictionary(dictionary_name, context);
|
||||
Poco::File(dictionary_metadata_tmp_path).remove();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOnDisk::removeTable(
|
||||
IDatabase & database,
|
||||
const Context & /* context */,
|
||||
@ -149,7 +329,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 +351,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 +400,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 +434,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 +468,25 @@ void DatabaseOnDisk::drop(const IDatabase & database, const Context & context)
|
||||
Poco::File(database.getMetadataPath()).remove(false);
|
||||
}
|
||||
|
||||
String DatabaseOnDisk::getTableMetadataPath(const IDatabase & database, const String & table_name)
|
||||
String DatabaseOnDisk::getObjectMetadataPath(const IDatabase & database, const String & table_name)
|
||||
{
|
||||
return detail::getTableMetadataPath(database.getMetadataPath(), table_name);
|
||||
return detail::getObjectMetadataPath(database.getMetadataPath(), table_name);
|
||||
}
|
||||
|
||||
time_t DatabaseOnDisk::getTableMetadataModificationTime(
|
||||
time_t DatabaseOnDisk::getObjectMetadataModificationTime(
|
||||
const IDatabase & database,
|
||||
const String & table_name)
|
||||
{
|
||||
String table_metadata_path = getTableMetadataPath(database, table_name);
|
||||
String table_metadata_path = getObjectMetadataPath(database, table_name);
|
||||
Poco::File meta_file(table_metadata_path);
|
||||
|
||||
if (meta_file.exists())
|
||||
{
|
||||
return meta_file.getLastModified().epochTime();
|
||||
}
|
||||
else
|
||||
{
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function)
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function)
|
||||
{
|
||||
Poco::DirectoryIterator dir_end;
|
||||
for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it)
|
||||
@ -265,11 +503,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(context.getPath() + 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(context.getPath() + 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
|
||||
{
|
||||
|
@ -13,12 +13,28 @@ namespace DB
|
||||
|
||||
namespace detail
|
||||
{
|
||||
String getTableMetadataPath(const String & base_path, const String & table_name);
|
||||
String getObjectMetadataPath(const String & base_path, const String & dictionary_name);
|
||||
String getDatabaseMetadataPath(const String & base_path);
|
||||
ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true);
|
||||
ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error);
|
||||
}
|
||||
|
||||
ASTPtr parseCreateQueryFromMetadataFile(const String & filepath, Poco::Logger * log);
|
||||
|
||||
std::pair<String, StoragePtr> createTableFromAST(
|
||||
ASTCreateQuery ast_create_query,
|
||||
const String & database_name,
|
||||
const String & database_data_path,
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag);
|
||||
|
||||
/** Get the row with the table definition based on the CREATE query.
|
||||
* It is an ATTACH query that you can execute to create a table from the correspondent database.
|
||||
* See the implementation.
|
||||
*/
|
||||
String getObjectDefinitionFromCreateQuery(const ASTPtr & query);
|
||||
|
||||
|
||||
/* Class to provide basic operations with tables when metadata is stored on disk in .sql files.
|
||||
*/
|
||||
class DatabaseOnDisk
|
||||
@ -31,12 +47,24 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query);
|
||||
|
||||
static void createDictionary(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query);
|
||||
|
||||
static void removeTable(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
Poco::Logger * log);
|
||||
|
||||
static void removeDictionary(
|
||||
IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
Poco::Logger * log);
|
||||
|
||||
template <typename Database>
|
||||
static void renameTable(
|
||||
IDatabase & database,
|
||||
@ -56,23 +84,33 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name);
|
||||
|
||||
static ASTPtr getCreateDictionaryQuery(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name);
|
||||
|
||||
static ASTPtr tryGetCreateDictionaryQuery(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name);
|
||||
|
||||
static ASTPtr getCreateDatabaseQuery(
|
||||
const IDatabase & database,
|
||||
const Context & context);
|
||||
|
||||
static void drop(const IDatabase & database, const Context & context);
|
||||
|
||||
static String getTableMetadataPath(
|
||||
static String getObjectMetadataPath(
|
||||
const IDatabase & database,
|
||||
const String & table_name);
|
||||
const String & object_name);
|
||||
|
||||
static time_t getTableMetadataModificationTime(
|
||||
static time_t getObjectMetadataModificationTime(
|
||||
const IDatabase & database,
|
||||
const String & table_name);
|
||||
const String & object_name);
|
||||
|
||||
|
||||
using IteratingFunction = std::function<void(const String &)>;
|
||||
static void iterateTableFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function);
|
||||
static void iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function);
|
||||
|
||||
private:
|
||||
static ASTPtr getCreateTableQueryImpl(
|
||||
@ -80,6 +118,12 @@ private:
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
bool throw_on_error);
|
||||
|
||||
static ASTPtr getCreateDictionaryQueryImpl(
|
||||
const IDatabase & database,
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
bool throw_on_error);
|
||||
};
|
||||
|
||||
|
||||
@ -126,7 +170,7 @@ void DatabaseOnDisk::renameTable(
|
||||
throw Exception{Exception::CreateFromPoco, e};
|
||||
}
|
||||
|
||||
ASTPtr ast = detail::getQueryFromMetadata(detail::getTableMetadataPath(database.getMetadataPath(), table_name));
|
||||
ASTPtr ast = detail::getQueryFromMetadata(detail::getObjectMetadataPath(database.getMetadataPath(), table_name));
|
||||
if (!ast)
|
||||
throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
ast->as<ASTCreateQuery &>().table = to_table_name;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <iomanip>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
@ -11,15 +10,18 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/Event.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -33,59 +35,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 +104,77 @@ DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_,
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOrdinary::loadTables(
|
||||
void DatabaseOrdinary::loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
using FileNames = std::vector<std::string>;
|
||||
FileNames file_names;
|
||||
|
||||
DatabaseOnDisk::iterateTableFiles(*this, log, context, [&file_names](const String & file_name)
|
||||
{
|
||||
file_names.push_back(file_name);
|
||||
});
|
||||
|
||||
if (file_names.empty())
|
||||
return;
|
||||
|
||||
/** Tables load faster if they are loaded in sorted (by name) order.
|
||||
* Otherwise (for the ext4 filesystem), `DirectoryIterator` iterates through them in some order,
|
||||
* which does not correspond to order tables creation and does not correspond to order of their location on disk.
|
||||
*/
|
||||
std::sort(file_names.begin(), file_names.end());
|
||||
using FileNames = std::map<std::string, ASTPtr>;
|
||||
FileNames file_names;
|
||||
|
||||
const size_t total_tables = file_names.size();
|
||||
LOG_INFO(log, "Total " << total_tables << " tables.");
|
||||
size_t total_dictionaries = 0;
|
||||
DatabaseOnDisk::iterateMetadataFiles(*this, log, context, [&file_names, &total_dictionaries, this](const String & file_name)
|
||||
{
|
||||
String full_path = metadata_path + "/" + file_name;
|
||||
try
|
||||
{
|
||||
auto ast = parseCreateQueryFromMetadataFile(full_path, log);
|
||||
if (ast)
|
||||
{
|
||||
auto * create_query = ast->as<ASTCreateQuery>();
|
||||
file_names[file_name] = ast;
|
||||
total_dictionaries += create_query->is_dictionary;
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot parse definition from metadata file " + full_path + ", error: " + e.displayText() + ", stack trace:\n"
|
||||
+ e.getStackTrace().toString(), ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
size_t total_tables = file_names.size() - total_dictionaries;
|
||||
|
||||
LOG_INFO(log, "Total " << total_tables << " tables and " << total_dictionaries << " dictionaries.");
|
||||
|
||||
AtomicStopwatch watch;
|
||||
std::atomic<size_t> tables_processed {0};
|
||||
std::atomic<size_t> tables_processed{0};
|
||||
std::atomic<size_t> dictionaries_processed{0};
|
||||
|
||||
auto loadOneTable = [&](const String & table)
|
||||
auto loadOneObject = [&](const ASTCreateQuery & query)
|
||||
{
|
||||
loadTable(context, getMetadataPath(), *this, getDatabaseName(), getDataPath(), table, has_force_restore_data_flag);
|
||||
loadObject(context, query, *this, getDataPath(), getDatabaseName(), has_force_restore_data_flag);
|
||||
|
||||
/// Messages, so that it's not boring to wait for the server to load for a long time.
|
||||
if (++tables_processed % PRINT_MESSAGE_EACH_N_TABLES == 0
|
||||
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
|
||||
watch.restart();
|
||||
}
|
||||
if (query.is_dictionary)
|
||||
logAboutProgress(log, ++dictionaries_processed, total_dictionaries, watch);
|
||||
else
|
||||
logAboutProgress(log, ++tables_processed, total_tables, watch);
|
||||
};
|
||||
|
||||
ThreadPool pool(SettingMaxThreads().getAutoValue());
|
||||
|
||||
for (const auto & file_name : file_names)
|
||||
for (const auto & name_with_query : file_names)
|
||||
{
|
||||
pool.scheduleOrThrowOnError([&]() { loadOneTable(file_name); });
|
||||
pool.scheduleOrThrowOnError([&]() { loadOneObject(name_with_query.second->as<const ASTCreateQuery &>()); });
|
||||
}
|
||||
|
||||
pool.wait();
|
||||
|
||||
/// After all tables was basically initialized, startup them.
|
||||
startupTables(pool);
|
||||
|
||||
/// Add database as repository
|
||||
auto dictionaries_repository = std::make_unique<ExternalLoaderDatabaseConfigRepository>(shared_from_this(), context);
|
||||
auto & external_loader = context.getExternalDictionariesLoader();
|
||||
external_loader.addConfigRepository(getDatabaseName(), std::move(dictionaries_repository));
|
||||
bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true);
|
||||
external_loader.reload(!lazy_load);
|
||||
}
|
||||
|
||||
|
||||
@ -160,18 +187,12 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
|
||||
return;
|
||||
|
||||
AtomicStopwatch watch;
|
||||
std::atomic<size_t> tables_processed {0};
|
||||
std::atomic<size_t> tables_processed{0};
|
||||
|
||||
auto startupOneTable = [&](const StoragePtr & table)
|
||||
{
|
||||
table->startup();
|
||||
|
||||
if (++tables_processed % PRINT_MESSAGE_EACH_N_TABLES == 0
|
||||
|| watch.compareAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
|
||||
{
|
||||
LOG_INFO(log, std::fixed << std::setprecision(2) << tables_processed * 100.0 / total_tables << "%");
|
||||
watch.restart();
|
||||
}
|
||||
logAboutProgress(log, ++tables_processed, total_tables, watch);
|
||||
};
|
||||
|
||||
try
|
||||
@ -187,7 +208,6 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
|
||||
thread_pool.wait();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOrdinary::createTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -197,6 +217,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 +232,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 +250,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 +267,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);
|
||||
@ -275,7 +320,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);
|
||||
@ -320,9 +365,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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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(const Context & context) 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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,18 +1,15 @@
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/StorageDictionary.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
#include <sstream>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -24,116 +21,119 @@ namespace ErrorCodes
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int DICTIONARY_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
|
||||
String getTableDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
namespace
|
||||
{
|
||||
ASTPtr query_clone = query->clone();
|
||||
auto & create = query_clone->as<ASTCreateQuery &>();
|
||||
|
||||
/// We remove everything that is not needed for ATTACH from the query.
|
||||
create.attach = true;
|
||||
create.database.clear();
|
||||
create.as_database.clear();
|
||||
create.as_table.clear();
|
||||
create.if_not_exists = false;
|
||||
create.is_populate = false;
|
||||
create.replace_view = false;
|
||||
|
||||
/// For views it is necessary to save the SELECT query itself, for the rest - on the contrary
|
||||
if (!create.is_view && !create.is_materialized_view && !create.is_live_view)
|
||||
create.select = nullptr;
|
||||
|
||||
create.format = nullptr;
|
||||
create.out_file = nullptr;
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
formatAST(create, statement_stream, false);
|
||||
statement_stream << '\n';
|
||||
return statement_stream.str();
|
||||
}
|
||||
|
||||
|
||||
std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
const String & definition,
|
||||
const String & database_name,
|
||||
const String & relative_db_path,
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag,
|
||||
const String & description_for_error_message)
|
||||
StoragePtr getDictionaryStorage(const Context & context, const String & table_name, const String & db_name)
|
||||
{
|
||||
ParserCreateQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), description_for_error_message, 0);
|
||||
|
||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.database = database_name;
|
||||
|
||||
if (ast_create_query.as_table_function)
|
||||
auto dict_name = db_name + "." + table_name;
|
||||
const auto & external_loader = context.getExternalDictionariesLoader();
|
||||
auto dict_ptr = external_loader.tryGetDictionary(dict_name);
|
||||
if (dict_ptr)
|
||||
{
|
||||
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
||||
return {ast_create_query.table, storage};
|
||||
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
|
||||
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
|
||||
return StorageDictionary::create(db_name, table_name, ColumnsDescription{columns}, context, true, dict_name);
|
||||
}
|
||||
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
||||
/// - the database has not been created yet;
|
||||
/// - the code is simpler, since the query is already brought to a suitable form.
|
||||
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
|
||||
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
|
||||
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context);
|
||||
ConstraintsDescription constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
||||
|
||||
return
|
||||
{
|
||||
ast_create_query.table,
|
||||
StorageFactory::instance().get(
|
||||
ast_create_query,
|
||||
relative_db_path + escapeForFileName(ast_create_query.table) + '/',
|
||||
ast_create_query.table, database_name, context, context.getGlobalContext(),
|
||||
columns, constraints,
|
||||
true, has_force_restore_data_flag)
|
||||
};
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
return tables.find(table_name) != tables.end() || dictionaries.find(table_name) != dictionaries.end();
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isDictionaryExist(
|
||||
const Context & /*context*/,
|
||||
const String & dictionary_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return dictionaries.find(dictionary_name) != dictionaries.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
return {};
|
||||
return it->second;
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it != tables.end())
|
||||
return it->second;
|
||||
}
|
||||
|
||||
if (isDictionaryExist(context, table_name))
|
||||
/// We don't need lock database here, because database doesn't store dictionary itself
|
||||
/// just metadata
|
||||
return getDictionaryStorage(context, table_name, getDatabaseName());
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
auto tables_it = getTablesIterator(context, filter_by_name);
|
||||
auto dictionaries_it = getDictionariesIterator(context, filter_by_name);
|
||||
|
||||
Tables result;
|
||||
while (tables_it && tables_it->isValid())
|
||||
{
|
||||
result.emplace(tables_it->name(), tables_it->table());
|
||||
tables_it->next();
|
||||
}
|
||||
|
||||
while (dictionaries_it && dictionaries_it->isValid())
|
||||
{
|
||||
auto table_name = dictionaries_it->name();
|
||||
auto table_ptr = getDictionaryStorage(context, table_name, getDatabaseName());
|
||||
if (table_ptr)
|
||||
result.emplace(table_name, table_ptr);
|
||||
dictionaries_it->next();
|
||||
}
|
||||
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(result);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!filter_by_table_name)
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
|
||||
|
||||
Tables filtered_tables;
|
||||
for (const auto & [table_name, storage] : tables)
|
||||
if (filter_by_table_name(table_name))
|
||||
filtered_tables.emplace(table_name, storage);
|
||||
return std::make_unique<DatabaseSnapshotIterator>(std::move(filtered_tables));
|
||||
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(filtered_tables));
|
||||
}
|
||||
|
||||
|
||||
DatabaseDictionariesIteratorPtr DatabaseWithOwnTablesBase::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!filter_by_dictionary_name)
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>(dictionaries);
|
||||
|
||||
Dictionaries filtered_dictionaries;
|
||||
for (const auto & dictionary_name : dictionaries)
|
||||
if (filter_by_dictionary_name(dictionary_name))
|
||||
filtered_dictionaries.emplace(dictionary_name);
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>(std::move(filtered_dictionaries));
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.empty();
|
||||
return tables.empty() && dictionaries.empty();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
@ -141,6 +141,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);
|
||||
@ -151,6 +154,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);
|
||||
@ -158,6 +176,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.
|
||||
@ -176,6 +213,7 @@ void DatabaseWithOwnTablesBase::shutdown()
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
tables.clear();
|
||||
dictionaries.clear();
|
||||
}
|
||||
|
||||
DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase()
|
||||
|
@ -16,62 +16,6 @@ namespace DB
|
||||
class Context;
|
||||
|
||||
|
||||
/** Get the row with the table definition based on the CREATE query.
|
||||
* It is an ATTACH query that you can execute to create a table from the correspondent database.
|
||||
* See the implementation.
|
||||
*/
|
||||
String getTableDefinitionFromCreateQuery(const ASTPtr & query);
|
||||
|
||||
|
||||
/** Create a table by its definition, without using InterpreterCreateQuery.
|
||||
* (InterpreterCreateQuery has more complex functionality, and it can not be used if the database has not been created yet)
|
||||
* Returns the table name and the table itself.
|
||||
* You must subsequently call IStorage::startup method to use the table.
|
||||
*/
|
||||
std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
const String & definition,
|
||||
const String & database_name,
|
||||
const String & relative_db_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 +24,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 +55,7 @@ protected:
|
||||
|
||||
mutable std::mutex mutex;
|
||||
Tables tables;
|
||||
Dictionaries dictionaries;
|
||||
|
||||
DatabaseWithOwnTablesBase(String name_) : name(std::move(name_)) { }
|
||||
};
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <ctime>
|
||||
@ -19,16 +20,14 @@ struct ConstraintsDescription;
|
||||
class ColumnsDescription;
|
||||
struct IndicesDescription;
|
||||
struct TableStructureWriteLockHolder;
|
||||
using Dictionaries = std::set<String>;
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
/** Allows to iterate over tables.
|
||||
*/
|
||||
class IDatabaseIterator
|
||||
class IDatabaseTablesIterator
|
||||
{
|
||||
public:
|
||||
virtual void next() = 0;
|
||||
@ -37,15 +36,57 @@ public:
|
||||
virtual const String & name() const = 0;
|
||||
virtual const StoragePtr & table() const = 0;
|
||||
|
||||
virtual ~IDatabaseIterator() {}
|
||||
virtual ~IDatabaseTablesIterator() = default;
|
||||
};
|
||||
|
||||
using DatabaseIteratorPtr = std::unique_ptr<IDatabaseIterator>;
|
||||
/// Copies list of tables and iterates through such snapshot.
|
||||
class DatabaseTablesSnapshotIterator : public IDatabaseTablesIterator
|
||||
{
|
||||
private:
|
||||
Tables tables;
|
||||
Tables::iterator it;
|
||||
|
||||
public:
|
||||
DatabaseTablesSnapshotIterator(Tables & tables_) : tables(tables_), it(tables.begin()) {}
|
||||
|
||||
DatabaseTablesSnapshotIterator(Tables && tables_) : tables(tables_), it(tables.begin()) {}
|
||||
|
||||
void next() { ++it; }
|
||||
|
||||
bool isValid() const { return it != tables.end(); }
|
||||
|
||||
const String & name() const { return it->first; }
|
||||
|
||||
const StoragePtr & table() const { return it->second; }
|
||||
};
|
||||
|
||||
/// Copies list of dictionaries and iterates through such snapshot.
|
||||
class DatabaseDictionariesSnapshotIterator
|
||||
{
|
||||
private:
|
||||
Dictionaries dictionaries;
|
||||
Dictionaries::iterator it;
|
||||
|
||||
public:
|
||||
DatabaseDictionariesSnapshotIterator() = default;
|
||||
DatabaseDictionariesSnapshotIterator(Dictionaries & dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
|
||||
|
||||
DatabaseDictionariesSnapshotIterator(Dictionaries && dictionaries_) : dictionaries(dictionaries_), it(dictionaries.begin()) {}
|
||||
|
||||
void next() { ++it; }
|
||||
|
||||
bool isValid() const { return !dictionaries.empty() && it != dictionaries.end(); }
|
||||
|
||||
const String & name() const { return *it; }
|
||||
};
|
||||
|
||||
using DatabaseTablesIteratorPtr = std::unique_ptr<IDatabaseTablesIterator>;
|
||||
using DatabaseDictionariesIteratorPtr = std::unique_ptr<DatabaseDictionariesSnapshotIterator>;
|
||||
|
||||
|
||||
/** Database engine.
|
||||
* It is responsible for:
|
||||
* - initialization of set of known tables;
|
||||
* - initialization of set of known tables and dictionaries;
|
||||
* - checking existence of a table and getting a table object;
|
||||
* - retrieving a list of all tables;
|
||||
* - creating and dropping tables;
|
||||
@ -60,7 +101,7 @@ public:
|
||||
|
||||
/// Load a set of existing tables.
|
||||
/// You can call only once, right after the object is created.
|
||||
virtual void loadTables(
|
||||
virtual void loadStoredObjects(
|
||||
Context & context,
|
||||
bool has_force_restore_data_flag) = 0;
|
||||
|
||||
@ -69,6 +110,11 @@ public:
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
|
||||
/// Check the existence of the dictionary
|
||||
virtual bool isDictionaryExist(
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
|
||||
/// Get the table for work. Return nullptr if there is no table.
|
||||
virtual StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
@ -78,7 +124,16 @@ public:
|
||||
|
||||
/// Get an iterator that allows you to pass through all the tables.
|
||||
/// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above.
|
||||
virtual DatabaseIteratorPtr getIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
|
||||
/// Get an iterator to pass through all the dictionaries.
|
||||
virtual DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name = {}) = 0;
|
||||
|
||||
/// Get an iterator to pass through all the tables and dictionary tables.
|
||||
virtual DatabaseTablesIteratorPtr getTablesWithDictionaryTablesIterator(const Context & context, const FilterByNameFunction & filter_by_name = {})
|
||||
{
|
||||
return getTablesIterator(context, filter_by_name);
|
||||
}
|
||||
|
||||
/// Is the database empty.
|
||||
virtual bool empty(const Context & context) const = 0;
|
||||
@ -90,17 +145,35 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) = 0;
|
||||
|
||||
/// Add the dictionary to the database. Record its presence in the metadata.
|
||||
virtual void createDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
const ASTPtr & query) = 0;
|
||||
|
||||
/// Delete the table from the database. Delete the metadata.
|
||||
virtual void removeTable(
|
||||
const Context & context,
|
||||
const String & name) = 0;
|
||||
|
||||
/// Delete the dictionary from the database. Delete the metadata.
|
||||
virtual void removeDictionary(
|
||||
const Context & context,
|
||||
const String & dictionary_name) = 0;
|
||||
|
||||
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
|
||||
virtual void attachTable(const String & name, const StoragePtr & table) = 0;
|
||||
|
||||
/// Add dictionary to the database, but do not add it to the metadata. The database may not support this method.
|
||||
/// load is false when we starting up and lazy_load is true, so we don't want to load dictionaries synchronously.
|
||||
virtual void attachDictionary(const String & name, const Context & context, bool reload = true) = 0;
|
||||
|
||||
/// Forget about the table without deleting it, and return it. The database may not support this method.
|
||||
virtual StoragePtr detachTable(const String & name) = 0;
|
||||
|
||||
/// Forget about the dictionary without deleting it, and return it. The database may not support this method.
|
||||
virtual void detachDictionary(const String & name, const Context & context, bool reload = true) = 0;
|
||||
|
||||
/// Rename the table and possibly move the table to another database.
|
||||
virtual void renameTable(
|
||||
const Context & /*context*/,
|
||||
@ -128,7 +201,7 @@ public:
|
||||
}
|
||||
|
||||
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.
|
||||
virtual time_t getTableMetadataModificationTime(
|
||||
virtual time_t getObjectMetadataModificationTime(
|
||||
const Context & context,
|
||||
const String & name) = 0;
|
||||
|
||||
@ -140,6 +213,14 @@ public:
|
||||
return tryGetCreateTableQuery(context, name);
|
||||
}
|
||||
|
||||
/// Get the CREATE DICTIONARY query for the dictionary. Returns nullptr if dictionary doesn't exists.
|
||||
virtual ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & name) const = 0;
|
||||
|
||||
virtual ASTPtr getCreateDictionaryQuery(const Context & context, const String & name) const
|
||||
{
|
||||
return tryGetCreateDictionaryQuery(context, name);
|
||||
}
|
||||
|
||||
/// Get the CREATE DATABASE query for current database.
|
||||
virtual ASTPtr getCreateDatabaseQuery(const Context & context) const = 0;
|
||||
|
||||
@ -150,7 +231,7 @@ public:
|
||||
/// Returns metadata path if the database supports it, empty string otherwise
|
||||
virtual String getMetadataPath() const { return {}; }
|
||||
/// Returns metadata path of a concrete table if the database supports it, empty string otherwise
|
||||
virtual String getTableMetadataPath(const String & /*table_name*/) const { return {}; }
|
||||
virtual String getObjectMetadataPath(const String & /*table_name*/) const { return {}; }
|
||||
|
||||
/// Ask all tables to complete the background threads they are using and delete all table objects.
|
||||
virtual void shutdown() = 0;
|
||||
|
@ -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})
|
||||
|
@ -611,7 +611,7 @@ void registerDictionaryCache(DictionaryFactory & factory)
|
||||
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
|
||||
return std::make_unique<CacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
|
||||
};
|
||||
factory.registerLayout("cache", create_layout);
|
||||
factory.registerLayout("cache", create_layout, false);
|
||||
}
|
||||
|
||||
|
||||
|
@ -52,7 +52,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Block & sample_block_,
|
||||
Context & context_)
|
||||
const Context & context_)
|
||||
: update_time{std::chrono::system_clock::from_time_t(0)}
|
||||
, dict_struct{dict_struct_}
|
||||
, host{config.getString(config_prefix + ".host")}
|
||||
@ -206,7 +206,7 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context) -> DictionarySourcePtr
|
||||
const Context & context) -> DictionarySourcePtr
|
||||
{
|
||||
return std::make_unique<ClickHouseDictionarySource>(dict_struct, config, config_prefix + ".clickhouse", sample_block, context);
|
||||
};
|
||||
|
@ -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);
|
||||
|
@ -415,7 +415,7 @@ void registerDictionaryComplexKeyCache(DictionaryFactory & factory)
|
||||
const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"};
|
||||
return std::make_unique<ComplexKeyCacheDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, size);
|
||||
};
|
||||
factory.registerLayout("complex_key_cache", create_layout);
|
||||
factory.registerLayout("complex_key_cache", create_layout, true);
|
||||
}
|
||||
|
||||
|
||||
|
@ -755,7 +755,7 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory)
|
||||
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
|
||||
return std::make_unique<ComplexKeyHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("complex_key_hashed", create_layout);
|
||||
factory.registerLayout("complex_key_hashed", create_layout, true);
|
||||
}
|
||||
|
||||
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <memory>
|
||||
#include "DictionarySourceFactory.h"
|
||||
#include "DictionaryStructure.h"
|
||||
#include "getDictionaryConfigurationFromAST.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -12,15 +13,21 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
|
||||
}
|
||||
|
||||
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout)
|
||||
void DictionaryFactory::registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex)
|
||||
{
|
||||
if (!registered_layouts.emplace(layout_type, std::move(create_layout)).second)
|
||||
throw Exception("DictionaryFactory: the layout name '" + layout_type + "' is not unique", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
layout_complexity[layout_type] = is_complex;
|
||||
|
||||
}
|
||||
|
||||
|
||||
DictionaryPtr DictionaryFactory::create(
|
||||
const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
const auto & layout_prefix = config_prefix + ".layout";
|
||||
@ -31,7 +38,7 @@ DictionaryPtr DictionaryFactory::create(
|
||||
|
||||
const DictionaryStructure dict_struct{config, config_prefix + ".structure"};
|
||||
|
||||
auto source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
|
||||
DictionarySourcePtr source_ptr = DictionarySourceFactory::instance().create(name, config, config_prefix + ".source", dict_struct, context);
|
||||
|
||||
const auto & layout_type = keys.front();
|
||||
|
||||
@ -39,14 +46,21 @@ DictionaryPtr DictionaryFactory::create(
|
||||
const auto found = registered_layouts.find(layout_type);
|
||||
if (found != registered_layouts.end())
|
||||
{
|
||||
const auto & create_layout = found->second;
|
||||
return create_layout(name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
const auto & layout_creator = found->second;
|
||||
return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr));
|
||||
}
|
||||
}
|
||||
|
||||
throw Exception{name + ": unknown dictionary layout type: " + layout_type, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
}
|
||||
|
||||
DictionaryPtr DictionaryFactory::create(const std::string & name, const ASTCreateQuery & ast, const Context & context) const
|
||||
{
|
||||
auto configurationFromAST = getDictionaryConfigurationFromAST(ast);
|
||||
return DictionaryFactory::create(name, *configurationFromAST, "dictionary", context);
|
||||
}
|
||||
|
||||
|
||||
DictionaryFactory & DictionaryFactory::instance()
|
||||
{
|
||||
static DictionaryFactory ret;
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include "IDictionary.h"
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
@ -27,7 +28,15 @@ public:
|
||||
|
||||
static DictionaryFactory & instance();
|
||||
|
||||
DictionaryPtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, Context & context) const;
|
||||
DictionaryPtr create(
|
||||
const std::string & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Context & context) const;
|
||||
|
||||
DictionaryPtr create(const std::string & name,
|
||||
const ASTCreateQuery & ast,
|
||||
const Context & context) const;
|
||||
|
||||
using Creator = std::function<DictionaryPtr(
|
||||
const std::string & name,
|
||||
@ -36,11 +45,15 @@ public:
|
||||
const std::string & config_prefix,
|
||||
DictionarySourcePtr source_ptr)>;
|
||||
|
||||
void registerLayout(const std::string & layout_type, Creator create_layout);
|
||||
bool isComplex(const std::string & layout_type) const { return layout_complexity.at(layout_type); }
|
||||
|
||||
void registerLayout(const std::string & layout_type, Creator create_layout, bool is_complex);
|
||||
|
||||
private:
|
||||
using LayoutRegistry = std::unordered_map<std::string, Creator>;
|
||||
LayoutRegistry registered_layouts;
|
||||
using LayoutComplexity = std::unordered_map<std::string, bool>;
|
||||
LayoutComplexity layout_complexity;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context)>;
|
||||
const Context & context)>;
|
||||
|
||||
DictionarySourceFactory();
|
||||
|
||||
@ -42,7 +42,7 @@ public:
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const DictionaryStructure & dict_struct,
|
||||
Context & context) const;
|
||||
const Context & context) const;
|
||||
|
||||
private:
|
||||
using SourceRegistry = std::unordered_map<std::string, Creator>;
|
||||
|
@ -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};
|
||||
|
@ -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};
|
||||
|
@ -724,7 +724,7 @@ void registerDictionaryFlat(DictionaryFactory & factory)
|
||||
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
|
||||
return std::make_unique<FlatDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("flat", create_layout);
|
||||
factory.registerLayout("flat", create_layout, false);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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};
|
||||
|
@ -787,8 +787,8 @@ void registerDictionaryHashed(DictionaryFactory & factory)
|
||||
const bool sparse = name == "sparse_hashed";
|
||||
return std::make_unique<HashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse);
|
||||
};
|
||||
factory.registerLayout("hashed", create_layout);
|
||||
factory.registerLayout("sparse_hashed", create_layout);
|
||||
factory.registerLayout("hashed", create_layout, false);
|
||||
factory.registerLayout("sparse_hashed", create_layout, false);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -691,7 +691,7 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory)
|
||||
const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false);
|
||||
return std::make_unique<RangeHashedDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("range_hashed", create_layout);
|
||||
factory.registerLayout("range_hashed", create_layout, false);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -767,7 +767,7 @@ void registerDictionaryTrie(DictionaryFactory & factory)
|
||||
// This is specialised trie for storing IPv4 and IPv6 prefixes.
|
||||
return std::make_unique<TrieDictionary>(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty);
|
||||
};
|
||||
factory.registerLayout("ip_trie", create_layout);
|
||||
factory.registerLayout("ip_trie", create_layout, true);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -238,7 +238,7 @@ void registerDictionarySourceXDBC(DictionarySourceFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
Block & sample_block,
|
||||
Context & context) -> DictionarySourcePtr {
|
||||
const Context & context) -> DictionarySourcePtr {
|
||||
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
|
||||
BridgeHelperPtr bridge = std::make_shared<XDBCBridgeHelper<ODBCBridgeMixin>>(
|
||||
context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string"));
|
||||
|
444
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.cpp
Normal file
444
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.cpp
Normal file
@ -0,0 +1,444 @@
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
|
||||
#include <Poco/DOM/AutoPtr.h>
|
||||
#include <Poco/DOM/Document.h>
|
||||
#include <Poco/DOM/Element.h>
|
||||
#include <Poco/DOM/Text.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DICTIONARY_DEFINITION;
|
||||
}
|
||||
|
||||
/// There are a lot of code, but it's very simple and straightforward
|
||||
/// We just convert
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Get value from field and convert it to string.
|
||||
/// Also remove quotes from strings.
|
||||
String getUnescapedFieldString(const Field & field)
|
||||
{
|
||||
String string = applyVisitor(FieldVisitorToString(), field);
|
||||
if (!string.empty() && string.front() == '\'' && string.back() == '\'')
|
||||
return string.substr(1, string.size() - 2);
|
||||
return string;
|
||||
}
|
||||
|
||||
|
||||
using namespace Poco;
|
||||
using namespace Poco::XML;
|
||||
/*
|
||||
* Transforms next definition
|
||||
* LIFETIME(MIN 10, MAX 100)
|
||||
* to the next configuration
|
||||
* <lifetime>
|
||||
* <min>10</min>
|
||||
* <max>100</max>
|
||||
* </lifetime>
|
||||
*/
|
||||
void buildLifetimeConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTDictionaryLifetime * lifetime)
|
||||
{
|
||||
|
||||
AutoPtr<Element> lifetime_element(doc->createElement("lifetime"));
|
||||
AutoPtr<Element> min_element(doc->createElement("min"));
|
||||
AutoPtr<Element> max_element(doc->createElement("max"));
|
||||
AutoPtr<Text> min_sec(doc->createTextNode(toString(lifetime->min_sec)));
|
||||
min_element->appendChild(min_sec);
|
||||
AutoPtr<Text> max_sec(doc->createTextNode(toString(lifetime->max_sec)));
|
||||
max_element->appendChild(max_sec);
|
||||
lifetime_element->appendChild(min_element);
|
||||
lifetime_element->appendChild(max_element);
|
||||
root->appendChild(lifetime_element);
|
||||
}
|
||||
|
||||
/*
|
||||
* Transforms next definition
|
||||
* LAYOUT(FLAT())
|
||||
* to the next configuration
|
||||
* <layout>
|
||||
* <flat/>
|
||||
* </layout>
|
||||
*
|
||||
* And next definition
|
||||
* LAYOUT(CACHE(SIZE_IN_CELLS 1000))
|
||||
* to the next one
|
||||
* <layout>
|
||||
* <cache>
|
||||
* <size_in_cells>1000</size_in_cells>
|
||||
* </cache>
|
||||
* </layout>
|
||||
*/
|
||||
void buildLayoutConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTDictionaryLayout * layout)
|
||||
{
|
||||
AutoPtr<Element> layout_element(doc->createElement("layout"));
|
||||
root->appendChild(layout_element);
|
||||
AutoPtr<Element> layout_type_element(doc->createElement(layout->layout_type));
|
||||
layout_element->appendChild(layout_type_element);
|
||||
if (layout->parameter.has_value())
|
||||
{
|
||||
const auto & param = layout->parameter;
|
||||
AutoPtr<Element> layout_type_parameter_element(doc->createElement(param->first));
|
||||
const ASTLiteral & literal = param->second->as<const ASTLiteral &>();
|
||||
AutoPtr<Text> value(doc->createTextNode(toString(literal.value.get<UInt64>())));
|
||||
layout_type_parameter_element->appendChild(value);
|
||||
layout_type_element->appendChild(layout_type_parameter_element);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Transforms next definition
|
||||
* RANGE(MIN StartDate, MAX EndDate)
|
||||
* to the next configuration
|
||||
* <range_min><name>StartDate</name></range_min>
|
||||
* <range_max><name>EndDate</name></range_max>
|
||||
*/
|
||||
void buildRangeConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTDictionaryRange * range)
|
||||
{
|
||||
// appends <key><name>value</name></key> to root
|
||||
auto appendElem = [&doc, &root](const std::string & key, const std::string & value)
|
||||
{
|
||||
AutoPtr<Element> element(doc->createElement(key));
|
||||
AutoPtr<Element> name(doc->createElement("name"));
|
||||
AutoPtr<Text> text(doc->createTextNode(value));
|
||||
name->appendChild(text);
|
||||
element->appendChild(name);
|
||||
root->appendChild(element);
|
||||
};
|
||||
|
||||
appendElem("range_min", range->min_attr_name);
|
||||
appendElem("range_max", range->max_attr_name);
|
||||
}
|
||||
|
||||
|
||||
/// Get primary key columns names from AST
|
||||
Names getPrimaryKeyColumns(const ASTExpressionList * primary_key)
|
||||
{
|
||||
Names result;
|
||||
const auto & children = primary_key->children;
|
||||
|
||||
for (size_t index = 0; index != children.size(); ++index)
|
||||
{
|
||||
const ASTIdentifier * key_part = children[index]->as<const ASTIdentifier>();
|
||||
result.push_back(key_part->name);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Transofrms single dictionary attribute to configuration
|
||||
* third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77
|
||||
* to
|
||||
* <attribute>
|
||||
* <name>third_column</name>
|
||||
* <type>UInt8</type>
|
||||
* <null_value>2</null_value>
|
||||
* <expression>(rand() % 100) * 77</expression>
|
||||
* </attribute>
|
||||
*/
|
||||
void buildSingleAttribute(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr)
|
||||
{
|
||||
AutoPtr<Element> attribute_element(doc->createElement("attribute"));
|
||||
root->appendChild(attribute_element);
|
||||
|
||||
AutoPtr<Element> name_element(doc->createElement("name"));
|
||||
AutoPtr<Text> name(doc->createTextNode(dict_attr->name));
|
||||
name_element->appendChild(name);
|
||||
attribute_element->appendChild(name_element);
|
||||
|
||||
AutoPtr<Element> type_element(doc->createElement("type"));
|
||||
AutoPtr<Text> type(doc->createTextNode(queryToString(dict_attr->type)));
|
||||
type_element->appendChild(type);
|
||||
attribute_element->appendChild(type_element);
|
||||
|
||||
AutoPtr<Element> null_value_element(doc->createElement("null_value"));
|
||||
String null_value_str;
|
||||
if (dict_attr->default_value)
|
||||
null_value_str = queryToString(dict_attr->default_value);
|
||||
AutoPtr<Text> null_value(doc->createTextNode(null_value_str));
|
||||
null_value_element->appendChild(null_value);
|
||||
attribute_element->appendChild(null_value_element);
|
||||
|
||||
if (dict_attr->expression != nullptr)
|
||||
{
|
||||
AutoPtr<Element> expression_element(doc->createElement("expression"));
|
||||
AutoPtr<Text> expression(doc->createTextNode(queryToString(dict_attr->expression)));
|
||||
expression_element->appendChild(expression);
|
||||
attribute_element->appendChild(expression_element);
|
||||
}
|
||||
|
||||
if (dict_attr->hierarchical)
|
||||
{
|
||||
AutoPtr<Element> hierarchical_element(doc->createElement("hierarchical"));
|
||||
AutoPtr<Text> hierarchical(doc->createTextNode("true"));
|
||||
hierarchical_element->appendChild(hierarchical);
|
||||
attribute_element->appendChild(hierarchical_element);
|
||||
}
|
||||
|
||||
if (dict_attr->injective)
|
||||
{
|
||||
AutoPtr<Element> injective_element(doc->createElement("injective"));
|
||||
AutoPtr<Text> injective(doc->createTextNode("true"));
|
||||
injective_element->appendChild(injective);
|
||||
attribute_element->appendChild(injective_element);
|
||||
}
|
||||
|
||||
if (dict_attr->is_object_id)
|
||||
{
|
||||
AutoPtr<Element> is_object_id_element(doc->createElement("is_object_id"));
|
||||
AutoPtr<Text> is_object_id(doc->createTextNode("true"));
|
||||
is_object_id_element->appendChild(is_object_id);
|
||||
attribute_element->appendChild(is_object_id_element);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Transforms
|
||||
* PRIMARY KEY Attr1 ,..., AttrN
|
||||
* to the next configuration
|
||||
* <id><name>Attr1</name></id>
|
||||
* or
|
||||
* <key>
|
||||
* <attribute>
|
||||
* <name>Attr1</name>
|
||||
* <type>UInt8</type>
|
||||
* </attribute>
|
||||
* ...
|
||||
* <attribute> fe
|
||||
* </key>
|
||||
*
|
||||
*/
|
||||
void buildPrimaryKeyConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
bool complex,
|
||||
const Names & key_names,
|
||||
const ASTExpressionList * dictionary_attributes)
|
||||
{
|
||||
if (!complex)
|
||||
{
|
||||
if (key_names.size() != 1)
|
||||
throw Exception("Primary key for simple dictionary must contain exactly one element",
|
||||
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
AutoPtr<Element> id_element(doc->createElement("id"));
|
||||
root->appendChild(id_element);
|
||||
AutoPtr<Element> name_element(doc->createElement("name"));
|
||||
id_element->appendChild(name_element);
|
||||
AutoPtr<Text> name(doc->createTextNode(*key_names.begin()));
|
||||
name_element->appendChild(name);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & children = dictionary_attributes->children;
|
||||
if (children.size() < key_names.size())
|
||||
throw Exception(
|
||||
"Primary key fields count is more, than dictionary attributes count.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
AutoPtr<Element> key_element(doc->createElement("key"));
|
||||
root->appendChild(key_element);
|
||||
for (const auto & key_name : key_names)
|
||||
{
|
||||
bool found = false;
|
||||
for (const auto & attr : children)
|
||||
{
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr = attr->as<const ASTDictionaryAttributeDeclaration>();
|
||||
if (dict_attr->name == key_name)
|
||||
{
|
||||
found = true;
|
||||
buildSingleAttribute(doc, key_element, dict_attr);
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!found)
|
||||
throw Exception(
|
||||
"Primary key field '" + key_name + "' not found among attributes.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Transforms list of ASTDictionaryAttributeDeclarations to list of dictionary attributes
|
||||
*/
|
||||
std::unordered_set<std::string> buildDictionaryAttributesConfiguration(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTExpressionList * dictionary_attributes,
|
||||
const Names & key_columns)
|
||||
{
|
||||
const auto & children = dictionary_attributes->children;
|
||||
std::unordered_set<std::string> dictionary_attributes_names;
|
||||
for (size_t i = 0; i < children.size(); ++i)
|
||||
{
|
||||
const ASTDictionaryAttributeDeclaration * dict_attr = children[i]->as<const ASTDictionaryAttributeDeclaration>();
|
||||
if (!dict_attr->type)
|
||||
throw Exception("Dictionary attribute must has type", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
dictionary_attributes_names.insert(dict_attr->name);
|
||||
if (std::find(key_columns.begin(), key_columns.end(), dict_attr->name) == key_columns.end())
|
||||
buildSingleAttribute(doc, root, dict_attr);
|
||||
}
|
||||
return dictionary_attributes_names;
|
||||
}
|
||||
|
||||
/** Transform function with key-value arguments to configuration
|
||||
* (used for source transformation)
|
||||
*/
|
||||
void buildConfigurationFromFunctionWithKeyValueArguments(
|
||||
AutoPtr<Document> doc,
|
||||
AutoPtr<Element> root,
|
||||
const ASTExpressionList * ast_expr_list)
|
||||
{
|
||||
const auto & children = ast_expr_list->children;
|
||||
for (size_t i = 0; i != children.size(); ++i)
|
||||
{
|
||||
const ASTPair * pair = children[i]->as<const ASTPair>();
|
||||
AutoPtr<Element> current_xml_element(doc->createElement(pair->first));
|
||||
root->appendChild(current_xml_element);
|
||||
|
||||
if (auto identifier = pair->second->as<const ASTIdentifier>(); identifier)
|
||||
{
|
||||
AutoPtr<Text> value(doc->createTextNode(identifier->name));
|
||||
current_xml_element->appendChild(value);
|
||||
}
|
||||
else if (auto literal = pair->second->as<const ASTLiteral>(); literal)
|
||||
{
|
||||
AutoPtr<Text> value(doc->createTextNode(getUnescapedFieldString(literal->value)));
|
||||
current_xml_element->appendChild(value);
|
||||
}
|
||||
else if (auto list = pair->second->as<const ASTExpressionList>(); list)
|
||||
{
|
||||
buildConfigurationFromFunctionWithKeyValueArguments(doc, current_xml_element, list);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
"Incorrect ASTPair contains wrong value, should be literal, identifier or list",
|
||||
ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Build source definition from ast.
|
||||
* SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))
|
||||
* to
|
||||
* <source>
|
||||
* <mysql>
|
||||
* <host>localhost</host>
|
||||
* ...
|
||||
* <replica>
|
||||
* <host>127.0.0.1</host>
|
||||
* ...
|
||||
* </replica>
|
||||
* </mysql>
|
||||
* </source>
|
||||
*/
|
||||
void buildSourceConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTFunctionWithKeyValueArguments * source)
|
||||
{
|
||||
AutoPtr<Element> outer_element(doc->createElement("source"));
|
||||
root->appendChild(outer_element);
|
||||
AutoPtr<Element> source_element(doc->createElement(source->name));
|
||||
outer_element->appendChild(source_element);
|
||||
buildConfigurationFromFunctionWithKeyValueArguments(doc, source_element, source->elements->as<const ASTExpressionList>());
|
||||
}
|
||||
|
||||
/** Check all AST fields are filled, throws exception
|
||||
* in other case
|
||||
*/
|
||||
void checkAST(const ASTCreateQuery & query)
|
||||
{
|
||||
if (!query.is_dictionary || query.dictionary == nullptr)
|
||||
throw Exception("Cannot convert dictionary to configuration from non-dictionary AST.", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary_attributes_list == nullptr || query.dictionary_attributes_list->children.empty())
|
||||
throw Exception("Cannot create dictionary with empty attributes list", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->layout == nullptr)
|
||||
throw Exception("Cannot create dictionary with empty layout", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->lifetime == nullptr)
|
||||
throw Exception("Cannot create dictionary with empty lifetime", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->primary_key == nullptr)
|
||||
throw Exception("Cannot create dictionary without primary key", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
if (query.dictionary->source == nullptr)
|
||||
throw Exception("Cannot create dictionary with empty source", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
|
||||
/// Range can be empty
|
||||
}
|
||||
|
||||
void checkPrimaryKey(const std::unordered_set<std::string> & all_attrs, const Names & key_attrs)
|
||||
{
|
||||
for (const auto & key_attr : key_attrs)
|
||||
if (all_attrs.count(key_attr) == 0)
|
||||
throw Exception("Unknown key attribute '" + key_attr + "'", ErrorCodes::INCORRECT_DICTIONARY_DEFINITION);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query)
|
||||
{
|
||||
checkAST(query);
|
||||
|
||||
AutoPtr<Poco::XML::Document> xml_document(new Poco::XML::Document());
|
||||
AutoPtr<Poco::XML::Element> document_root(xml_document->createElement("dictionaries"));
|
||||
xml_document->appendChild(document_root);
|
||||
AutoPtr<Poco::XML::Element> current_dictionary(xml_document->createElement("dictionary"));
|
||||
document_root->appendChild(current_dictionary);
|
||||
AutoPtr<Poco::Util::XMLConfiguration> conf(new Poco::Util::XMLConfiguration());
|
||||
|
||||
AutoPtr<Poco::XML::Element> name_element(xml_document->createElement("name"));
|
||||
current_dictionary->appendChild(name_element);
|
||||
AutoPtr<Text> name(xml_document->createTextNode(query.database + "." + query.table));
|
||||
name_element->appendChild(name);
|
||||
|
||||
AutoPtr<Element> structure_element(xml_document->createElement("structure"));
|
||||
current_dictionary->appendChild(structure_element);
|
||||
Names pk_attrs = getPrimaryKeyColumns(query.dictionary->primary_key);
|
||||
auto dictionary_layout = query.dictionary->layout;
|
||||
|
||||
bool complex = DictionaryFactory::instance().isComplex(dictionary_layout->layout_type);
|
||||
|
||||
auto all_attr_names = buildDictionaryAttributesConfiguration(xml_document, structure_element, query.dictionary_attributes_list, pk_attrs);
|
||||
checkPrimaryKey(all_attr_names, pk_attrs);
|
||||
|
||||
buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list);
|
||||
|
||||
buildLayoutConfiguration(xml_document, current_dictionary, dictionary_layout);
|
||||
buildSourceConfiguration(xml_document, current_dictionary, query.dictionary->source);
|
||||
buildLifetimeConfiguration(xml_document, current_dictionary, query.dictionary->lifetime);
|
||||
|
||||
if (query.dictionary->range)
|
||||
buildRangeConfiguration(xml_document, structure_element, query.dictionary->range);
|
||||
|
||||
conf->load(xml_document);
|
||||
return conf;
|
||||
}
|
||||
|
||||
}
|
15
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.h
Normal file
15
dbms/src/Dictionaries/getDictionaryConfigurationFromAST.h
Normal file
@ -0,0 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
using DictionaryConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
|
||||
|
||||
/// Convert dictionary AST to Poco::AbstractConfiguration
|
||||
/// This function is necessary because all loadable objects configuration are Poco::AbstractConfiguration
|
||||
/// Can throw exception if query is ill-formed
|
||||
DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuery & query);
|
||||
|
||||
}
|
224
dbms/src/Dictionaries/tests/gtest_dictionary_configuration.cpp
Normal file
224
dbms/src/Dictionaries/tests/gtest_dictionary_configuration.cpp
Normal file
@ -0,0 +1,224 @@
|
||||
#include <iostream>
|
||||
|
||||
#include <sstream>
|
||||
#include <Core/Types.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserDictionary.h>
|
||||
#include <Parsers/ParserDropQuery.h>
|
||||
#include <Parsers/ParserTablePropertiesQuery.h>
|
||||
#include <Parsers/TablePropertiesQueriesASTs.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
#include <Dictionaries/registerDictionaries.h>
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
static bool registered = false;
|
||||
/// For debug
|
||||
std::string configurationToString(const DictionaryConfigurationPtr & config)
|
||||
{
|
||||
const Poco::Util::XMLConfiguration * xml_config = dynamic_cast<const Poco::Util::XMLConfiguration *>(config.get());
|
||||
std::ostringstream oss;
|
||||
xml_config->save(oss);
|
||||
return oss.str();
|
||||
}
|
||||
|
||||
TEST(ConvertDictionaryAST, SimpleDictConfiguration)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY test.dict1"
|
||||
" ("
|
||||
" key_column UInt64 DEFAULT 0,"
|
||||
" second_column UInt8 DEFAULT 1,"
|
||||
" third_column UInt8 DEFAULT 2"
|
||||
" )"
|
||||
" PRIMARY KEY key_column"
|
||||
" SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' PASSWORD '' DB 'test' TABLE 'table_for_dict'))"
|
||||
" LAYOUT(FLAT())"
|
||||
" LIFETIME(MIN 1 MAX 10)"
|
||||
" RANGE(MIN second_column MAX third_column)";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
|
||||
/// name
|
||||
EXPECT_EQ(config->getString("dictionary.name"), "test.dict1");
|
||||
|
||||
/// lifetime
|
||||
EXPECT_EQ(config->getInt("dictionary.lifetime.min"), 1);
|
||||
EXPECT_EQ(config->getInt("dictionary.lifetime.max"), 10);
|
||||
|
||||
/// range
|
||||
EXPECT_EQ(config->getString("dictionary.structure.range_min"), "second_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure.range_max"), "third_column");
|
||||
|
||||
/// source
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.host"), "localhost");
|
||||
EXPECT_EQ(config->getInt("dictionary.source.clickhouse.port"), 9000);
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.user"), "default");
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.password"), "");
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.db"), "test");
|
||||
EXPECT_EQ(config->getString("dictionary.source.clickhouse.table"), "table_for_dict");
|
||||
|
||||
/// attributes and key
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config->keys("dictionary.structure", keys);
|
||||
|
||||
EXPECT_EQ(keys.size(), 5); /// + ranged keys
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".name"), "second_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getInt("dictionary.structure." + keys[0] + ".null_value"), 1);
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".name"), "third_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getInt("dictionary.structure." + keys[1] + ".null_value"), 2);
|
||||
|
||||
EXPECT_EQ(keys[2], "id");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[2] + ".name"), "key_column");
|
||||
|
||||
/// layout
|
||||
EXPECT_TRUE(config->has("dictionary.layout.flat"));
|
||||
}
|
||||
|
||||
|
||||
TEST(ConvertDictionaryAST, TrickyAttributes)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY dict2"
|
||||
" ("
|
||||
" key_column UInt64 IS_OBJECT_ID,"
|
||||
" second_column UInt8 HIERARCHICAL INJECTIVE,"
|
||||
" third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77"
|
||||
" )"
|
||||
" PRIMARY KEY key_column"
|
||||
" LAYOUT(hashed())"
|
||||
" LIFETIME(MIN 1 MAX 10)"
|
||||
" SOURCE(CLICKHOUSE(HOST 'localhost'))";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config->keys("dictionary.structure", keys);
|
||||
|
||||
EXPECT_EQ(keys.size(), 3);
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".name"), "second_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".null_value"), "");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".hierarchical"), "true");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[0] + ".injective"), "true");
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".name"), "third_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".type"), "UInt8");
|
||||
EXPECT_EQ(config->getInt("dictionary.structure." + keys[1] + ".null_value"), 2);
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[1] + ".expression"), "(rand() % 100) * 77");
|
||||
|
||||
EXPECT_EQ(keys[2], "id");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + keys[2] + ".name"), "key_column");
|
||||
}
|
||||
|
||||
|
||||
TEST(ConvertDictionaryAST, ComplexKeyAndLayoutWithParams)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY dict4"
|
||||
" ("
|
||||
" key_column1 String,"
|
||||
" key_column2 UInt64,"
|
||||
" third_column UInt8,"
|
||||
" fourth_column UInt8"
|
||||
" )"
|
||||
" PRIMARY KEY key_column1, key_column2"
|
||||
" SOURCE(MYSQL())"
|
||||
" LAYOUT(COMPLEX_KEY_CACHE(size_in_cells 50))"
|
||||
" LIFETIME(MIN 1 MAX 10)";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config->keys("dictionary.structure.key", keys);
|
||||
|
||||
EXPECT_EQ(keys.size(), 2);
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[0] + ".name"), "key_column1");
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[0] + ".type"), "String");
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[1] + ".name"), "key_column2");
|
||||
EXPECT_EQ(config->getString("dictionary.structure.key." + keys[1] + ".type"), "UInt64");
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys attrs;
|
||||
config->keys("dictionary.structure", attrs);
|
||||
|
||||
EXPECT_EQ(attrs.size(), 3);
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[0] + ".name"), "third_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[0] + ".type"), "UInt8");
|
||||
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[1] + ".name"), "fourth_column");
|
||||
EXPECT_EQ(config->getString("dictionary.structure." + attrs[1] + ".type"), "UInt8");
|
||||
|
||||
EXPECT_EQ(attrs[2], "key");
|
||||
|
||||
EXPECT_EQ(config->getInt("dictionary.layout.complex_key_cache.size_in_cells"), 50);
|
||||
}
|
||||
|
||||
|
||||
TEST(ConvertDictionaryAST, ComplexSource)
|
||||
{
|
||||
if (!registered)
|
||||
{
|
||||
registerDictionaries();
|
||||
registered = true;
|
||||
}
|
||||
|
||||
String input = " CREATE DICTIONARY dict4"
|
||||
" ("
|
||||
" key_column UInt64,"
|
||||
" second_column UInt8,"
|
||||
" third_column UInt8"
|
||||
" )"
|
||||
" PRIMARY KEY key_column"
|
||||
" SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA(HOST '127.0.0.1' PRIORITY 1) PASSWORD ''))"
|
||||
" LAYOUT(CACHE(size_in_cells 50))"
|
||||
" LIFETIME(MIN 1 MAX 10)"
|
||||
" RANGE(MIN second_column MAX third_column)";
|
||||
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
DictionaryConfigurationPtr config = getDictionaryConfigurationFromAST(*create);
|
||||
/// source
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.host"), "localhost");
|
||||
EXPECT_EQ(config->getInt("dictionary.source.mysql.port"), 9000);
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.user"), "default");
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.password"), "");
|
||||
EXPECT_EQ(config->getString("dictionary.source.mysql.replica.host"), "127.0.0.1");
|
||||
EXPECT_EQ(config->getInt("dictionary.source.mysql.replica.priority"), 1);
|
||||
}
|
146
dbms/src/Functions/CRC.cpp
Normal file
146
dbms/src/Functions/CRC.cpp
Normal file
@ -0,0 +1,146 @@
|
||||
#include <zlib.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringOrArrayToT.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <class T>
|
||||
struct CRCBase
|
||||
{
|
||||
T tab[256];
|
||||
CRCBase(T polynomial)
|
||||
{
|
||||
for (size_t i = 0; i < 256; ++i)
|
||||
{
|
||||
T c = i;
|
||||
for (size_t j = 0; j < 8; ++j)
|
||||
{
|
||||
c = c & 1 ? polynomial ^ (c >> 1) : c >> 1;
|
||||
}
|
||||
tab[i] = c;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <class T, T polynomial>
|
||||
struct CRCImpl
|
||||
{
|
||||
using ReturnType = T;
|
||||
|
||||
static T make_crc(const unsigned char *buf, size_t size)
|
||||
{
|
||||
static CRCBase<ReturnType> base(polynomial);
|
||||
|
||||
T i, crc;
|
||||
|
||||
crc = 0;
|
||||
for (i = 0; i < size; i++)
|
||||
{
|
||||
crc = base.tab[(crc ^ buf[i]) & 0xff] ^ (crc >> 8);
|
||||
}
|
||||
return crc;
|
||||
}
|
||||
};
|
||||
|
||||
static constexpr UInt64 CRC64_ECMA = 0xc96c5795d7870f42ULL;
|
||||
struct CRC64ECMAImpl : public CRCImpl<UInt64, CRC64_ECMA>
|
||||
{
|
||||
static constexpr auto name = "CRC64";
|
||||
};
|
||||
|
||||
static constexpr UInt32 CRC32_IEEE = 0xedb88320;
|
||||
struct CRC32IEEEImpl : public CRCImpl<UInt32, CRC32_IEEE>
|
||||
{
|
||||
static constexpr auto name = "CRC32IEEE";
|
||||
};
|
||||
|
||||
struct CRC32ZLIBImpl
|
||||
{
|
||||
using ReturnType = UInt32;
|
||||
static constexpr auto name = "CRC32";
|
||||
|
||||
static UInt32 make_crc(const unsigned char *buf, size_t size)
|
||||
{ return crc32_z(0L, buf, size); }
|
||||
};
|
||||
|
||||
} // \anonymous
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
template <class Impl>
|
||||
struct CRCFunctionWrapper
|
||||
{
|
||||
static constexpr auto is_fixed_to_constant = true;
|
||||
using ReturnType = typename Impl::ReturnType;
|
||||
|
||||
static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<ReturnType> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = do_crc(data, prev_offset, offsets[i] - prev_offset - 1);
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_fixed_to_constant(const ColumnString::Chars & data, size_t n, ReturnType & res) { res = do_crc(data, 0, n); }
|
||||
|
||||
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<ReturnType> & res)
|
||||
{
|
||||
size_t size = data.size() / n;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = do_crc(data, i * n, n);
|
||||
}
|
||||
}
|
||||
|
||||
[[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray<ReturnType> & /*res*/)
|
||||
{
|
||||
throw Exception("Cannot apply function " + std::string(Impl::name) + " to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
private:
|
||||
static ReturnType do_crc(const ColumnString::Chars & buf, size_t offset, size_t size)
|
||||
{
|
||||
const unsigned char * p = reinterpret_cast<const unsigned char *>(&buf[0]) + offset;
|
||||
return Impl::make_crc(p, size);
|
||||
}
|
||||
};
|
||||
|
||||
template <class T>
|
||||
using FunctionCRC = FunctionStringOrArrayToT<CRCFunctionWrapper<T>, T, typename T::ReturnType>;
|
||||
// The same as IEEE variant, but uses 0xffffffff as initial value
|
||||
// This is the default
|
||||
//
|
||||
// (And zlib is used here, since it has optimized version)
|
||||
using FunctionCRC32ZLIB = FunctionCRC<CRC32ZLIBImpl>;
|
||||
// Uses CRC-32-IEEE 802.3 polynomial
|
||||
using FunctionCRC32IEEE = FunctionCRC<CRC32IEEEImpl>;
|
||||
// Uses CRC-64-ECMA polynomial
|
||||
using FunctionCRC64ECMA = FunctionCRC<CRC64ECMAImpl>;
|
||||
|
||||
template <class T>
|
||||
void registerFunctionCRCImpl(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<T>(T::name, FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
void registerFunctionCRC(FunctionFactory & factory)
|
||||
{
|
||||
registerFunctionCRCImpl<FunctionCRC32ZLIB>(factory);
|
||||
registerFunctionCRCImpl<FunctionCRC32IEEE>(factory);
|
||||
registerFunctionCRCImpl<FunctionCRC64ECMA>(factory);
|
||||
}
|
||||
|
||||
}
|
@ -1,68 +0,0 @@
|
||||
#include <zlib.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionStringOrArrayToT.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
/** Calculates the CRC32 of a string
|
||||
*/
|
||||
struct CRC32Impl
|
||||
{
|
||||
static constexpr auto is_fixed_to_constant = true;
|
||||
|
||||
static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray<UInt32> & res)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
|
||||
ColumnString::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = do_crc32(data, prev_offset, offsets[i] - prev_offset - 1);
|
||||
prev_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
|
||||
static void vector_fixed_to_constant(const ColumnString::Chars & data, size_t n, UInt32 & res) { res = do_crc32(data, 0, n); }
|
||||
|
||||
static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt32> & res)
|
||||
{
|
||||
size_t size = data.size() / n;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
res[i] = do_crc32(data, i * n, n);
|
||||
}
|
||||
}
|
||||
|
||||
[[noreturn]] static void array(const ColumnString::Offsets & /*offsets*/, PaddedPODArray<UInt32> & /*res*/)
|
||||
{
|
||||
throw Exception("Cannot apply function CRC32 to Array argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
private:
|
||||
static uint32_t do_crc32(const ColumnString::Chars & buf, size_t offset, size_t size)
|
||||
{
|
||||
const unsigned char * p = reinterpret_cast<const unsigned char *>(&buf[0]) + offset;
|
||||
return crc32(0L, p, size);
|
||||
}
|
||||
};
|
||||
|
||||
struct NameCRC32
|
||||
{
|
||||
static constexpr auto name = "CRC32";
|
||||
};
|
||||
using FunctionCRC32 = FunctionStringOrArrayToT<CRC32Impl, NameCRC32, UInt32>;
|
||||
|
||||
void registerFunctionCRC32(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionCRC32>(NameCRC32::name, FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -8,6 +8,7 @@ namespace DB
|
||||
void registerFunctionsJSON(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionJSON<NameJSONHas, JSONHasImpl>>();
|
||||
factory.registerFunction<FunctionJSON<NameIsValidJSON, IsValidJSONImpl>>();
|
||||
factory.registerFunction<FunctionJSON<NameJSONLength, JSONLengthImpl>>();
|
||||
factory.registerFunction<FunctionJSON<NameJSONKey, JSONKeyImpl>>();
|
||||
factory.registerFunction<FunctionJSON<NameJSONType, JSONTypeImpl>>();
|
||||
|
@ -279,6 +279,7 @@ private:
|
||||
|
||||
|
||||
struct NameJSONHas { static constexpr auto name{"JSONHas"}; };
|
||||
struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; };
|
||||
struct NameJSONLength { static constexpr auto name{"JSONLength"}; };
|
||||
struct NameJSONKey { static constexpr auto name{"JSONKey"}; };
|
||||
struct NameJSONType { static constexpr auto name{"JSONType"}; };
|
||||
@ -311,6 +312,36 @@ public:
|
||||
};
|
||||
|
||||
|
||||
template <typename JSONParser>
|
||||
class IsValidJSONImpl
|
||||
{
|
||||
public:
|
||||
static DataTypePtr getType(const char * function_name, const ColumnsWithTypeAndName & arguments)
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
{
|
||||
/// IsValidJSON() shouldn't get parameters other than JSON.
|
||||
throw Exception{"Function " + String(function_name) + " needs exactly one argument",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
}
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
}
|
||||
|
||||
using Iterator = typename JSONParser::Iterator;
|
||||
static bool addValueToColumn(IColumn & dest, const Iterator &)
|
||||
{
|
||||
/// This function is called only if JSON is valid.
|
||||
/// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function.
|
||||
ColumnVector<UInt8> & col_vec = assert_cast<ColumnVector<UInt8> &>(dest);
|
||||
col_vec.insertValue(1);
|
||||
return true;
|
||||
}
|
||||
|
||||
static constexpr size_t num_extra_arguments = 0;
|
||||
static void prepare(const char *, const Block &, const ColumnNumbers &, size_t) {}
|
||||
};
|
||||
|
||||
|
||||
template <typename JSONParser>
|
||||
class JSONLengthImpl
|
||||
{
|
||||
|
68
dbms/src/Functions/getScalar.cpp
Normal file
68
dbms/src/Functions/getScalar.cpp
Normal file
@ -0,0 +1,68 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Common/Macros.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
/** Get scalar value of sub queries from query context via IAST::Hash.
|
||||
*/
|
||||
class FunctionGetScalar : public IFunction
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "__getScalar";
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<FunctionGetScalar>(context);
|
||||
}
|
||||
|
||||
FunctionGetScalar(const Context & context_) : context(context_) {}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 1;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (arguments.size() != 1 || !isString(arguments[0].type) || !isColumnConst(*arguments[0].column))
|
||||
throw Exception("Function " + getName() + " accepts one const string argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
auto scalar_name = assert_cast<const ColumnConst &>(*arguments[0].column).getField().get<String>();
|
||||
scalar = context.getScalar(scalar_name).getByPosition(0);
|
||||
return scalar.type;
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
block.getByPosition(result).column = ColumnConst::create(scalar.column, input_rows_count);
|
||||
}
|
||||
|
||||
private:
|
||||
mutable ColumnWithTypeAndName scalar;
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionGetScalar(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionGetScalar>();
|
||||
}
|
||||
|
||||
}
|
@ -52,6 +52,7 @@ void registerFunctionEvalMLMethod(FunctionFactory &);
|
||||
void registerFunctionBasename(FunctionFactory &);
|
||||
void registerFunctionTransform(FunctionFactory &);
|
||||
void registerFunctionGetMacro(FunctionFactory &);
|
||||
void registerFunctionGetScalar(FunctionFactory &);
|
||||
|
||||
#if USE_ICU
|
||||
void registerFunctionConvertCharset(FunctionFactory &);
|
||||
@ -106,6 +107,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
registerFunctionBasename(factory);
|
||||
registerFunctionTransform(factory);
|
||||
registerFunctionGetMacro(factory);
|
||||
registerFunctionGetScalar(factory);
|
||||
|
||||
#if USE_ICU
|
||||
registerFunctionConvertCharset(factory);
|
||||
|
@ -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);
|
||||
|
@ -23,7 +23,7 @@ template <typename F>
|
||||
inline void forEachTable(Context & context, F && f)
|
||||
{
|
||||
for (auto & elem : context.getDatabases())
|
||||
for (auto iterator = elem.second->getIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
f(iterator->table());
|
||||
|
||||
}
|
||||
|
@ -29,7 +29,7 @@
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/Set.h>
|
||||
|
@ -11,6 +11,9 @@
|
||||
#include <Common/HashTable/FixedHashMap.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/TwoLevelHashMap.h>
|
||||
#include <Common/HashTable/StringHashMap.h>
|
||||
#include <Common/HashTable/TwoLevelStringHashMap.h>
|
||||
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/UInt128.h>
|
||||
#include <Common/LRUCache.h>
|
||||
@ -69,12 +72,20 @@ using AggregatedDataWithUInt8Key = FixedHashMap<UInt8, AggregateDataPtr>;
|
||||
using AggregatedDataWithUInt16Key = FixedHashMap<UInt16, AggregateDataPtr>;
|
||||
|
||||
using AggregatedDataWithUInt64Key = HashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
|
||||
|
||||
using AggregatedDataWithShortStringKey = StringHashMap<AggregateDataPtr>;
|
||||
|
||||
using AggregatedDataWithStringKey = HashMapWithSavedHash<StringRef, AggregateDataPtr>;
|
||||
|
||||
using AggregatedDataWithKeys128 = HashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
|
||||
using AggregatedDataWithKeys256 = HashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
|
||||
|
||||
using AggregatedDataWithUInt64KeyTwoLevel = TwoLevelHashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
|
||||
|
||||
using AggregatedDataWithShortStringKeyTwoLevel = TwoLevelStringHashMap<AggregateDataPtr>;
|
||||
|
||||
using AggregatedDataWithStringKeyTwoLevel = TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr>;
|
||||
|
||||
using AggregatedDataWithKeys128TwoLevel = TwoLevelHashMap<UInt128, AggregateDataPtr, UInt128HashCRC32>;
|
||||
using AggregatedDataWithKeys256TwoLevel = TwoLevelHashMap<UInt256, AggregateDataPtr, UInt256HashCRC32>;
|
||||
|
||||
@ -139,6 +150,8 @@ struct AggregationDataWithNullKeyTwoLevel : public Base
|
||||
|
||||
template <typename ... Types>
|
||||
using HashTableWithNullKey = AggregationDataWithNullKey<HashMapTable<Types ...>>;
|
||||
template <typename ... Types>
|
||||
using StringHashTableWithNullKey = AggregationDataWithNullKey<StringHashMap<Types ...>>;
|
||||
|
||||
using AggregatedDataWithNullableUInt8Key = AggregationDataWithNullKey<AggregatedDataWithUInt8Key>;
|
||||
using AggregatedDataWithNullableUInt16Key = AggregationDataWithNullKey<AggregatedDataWithUInt16Key>;
|
||||
@ -149,6 +162,10 @@ using AggregatedDataWithNullableStringKey = AggregationDataWithNullKey<Aggregate
|
||||
using AggregatedDataWithNullableUInt64KeyTwoLevel = AggregationDataWithNullKeyTwoLevel<
|
||||
TwoLevelHashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>,
|
||||
TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>;
|
||||
|
||||
using AggregatedDataWithNullableShortStringKeyTwoLevel = AggregationDataWithNullKeyTwoLevel<
|
||||
TwoLevelStringHashMap<AggregateDataPtr, HashTableAllocator, StringHashTableWithNullKey>>;
|
||||
|
||||
using AggregatedDataWithNullableStringKeyTwoLevel = AggregationDataWithNullKeyTwoLevel<
|
||||
TwoLevelHashMapWithSavedHash<StringRef, AggregateDataPtr, DefaultHash<StringRef>,
|
||||
TwoLevelHashTableGrower<>, HashTableAllocator, HashTableWithNullKey>>;
|
||||
@ -216,6 +233,32 @@ struct AggregationMethodString
|
||||
};
|
||||
|
||||
|
||||
/// Same as above but without cache
|
||||
template <typename TData>
|
||||
struct AggregationMethodStringNoCache
|
||||
{
|
||||
using Data = TData;
|
||||
using Key = typename Data::key_type;
|
||||
using Mapped = typename Data::mapped_type;
|
||||
|
||||
Data data;
|
||||
|
||||
AggregationMethodStringNoCache() {}
|
||||
|
||||
template <typename Other>
|
||||
AggregationMethodStringNoCache(const Other & other) : data(other.data) {}
|
||||
|
||||
using State = ColumnsHashing::HashMethodString<typename Data::value_type, Mapped, true, false>;
|
||||
|
||||
static const bool low_cardinality_optimization = false;
|
||||
|
||||
static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &)
|
||||
{
|
||||
key_columns[0]->insertData(key.data, key.size);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// For the case where there is one fixed-length string key.
|
||||
template <typename TData>
|
||||
struct AggregationMethodFixedString
|
||||
@ -241,6 +284,32 @@ struct AggregationMethodFixedString
|
||||
}
|
||||
};
|
||||
|
||||
/// Same as above but without cache
|
||||
template <typename TData>
|
||||
struct AggregationMethodFixedStringNoCache
|
||||
{
|
||||
using Data = TData;
|
||||
using Key = typename Data::key_type;
|
||||
using Mapped = typename Data::mapped_type;
|
||||
|
||||
Data data;
|
||||
|
||||
AggregationMethodFixedStringNoCache() {}
|
||||
|
||||
template <typename Other>
|
||||
AggregationMethodFixedStringNoCache(const Other & other) : data(other.data) {}
|
||||
|
||||
using State = ColumnsHashing::HashMethodFixedString<typename Data::value_type, Mapped, true, false>;
|
||||
|
||||
static const bool low_cardinality_optimization = false;
|
||||
|
||||
static void insertKeyIntoColumns(const StringRef & key, MutableColumns & key_columns, const Sizes &)
|
||||
{
|
||||
key_columns[0]->insertData(key.data, key.size);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Single low cardinality column.
|
||||
template <typename SingleColumnMethod>
|
||||
struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
|
||||
@ -434,16 +503,16 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
|
||||
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64Key>> key32;
|
||||
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64Key>> key64;
|
||||
std::unique_ptr<AggregationMethodString<AggregatedDataWithStringKey>> key_string;
|
||||
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKey>> key_fixed_string;
|
||||
std::unique_ptr<AggregationMethodStringNoCache<AggregatedDataWithShortStringKey>> key_string;
|
||||
std::unique_ptr<AggregationMethodFixedStringNoCache<AggregatedDataWithShortStringKey>> key_fixed_string;
|
||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128>> keys128;
|
||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256;
|
||||
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKey>> serialized;
|
||||
|
||||
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64KeyTwoLevel>> key32_two_level;
|
||||
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyTwoLevel>> key64_two_level;
|
||||
std::unique_ptr<AggregationMethodString<AggregatedDataWithStringKeyTwoLevel>> key_string_two_level;
|
||||
std::unique_ptr<AggregationMethodFixedString<AggregatedDataWithStringKeyTwoLevel>> key_fixed_string_two_level;
|
||||
std::unique_ptr<AggregationMethodStringNoCache<AggregatedDataWithShortStringKeyTwoLevel>> key_string_two_level;
|
||||
std::unique_ptr<AggregationMethodFixedStringNoCache<AggregatedDataWithShortStringKeyTwoLevel>> key_fixed_string_two_level;
|
||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys128TwoLevel>> keys128_two_level;
|
||||
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel>> keys256_two_level;
|
||||
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyTwoLevel>> serialized_two_level;
|
||||
|
@ -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();
|
||||
|
@ -33,11 +33,13 @@ SelectStreamFactory::SelectStreamFactory(
|
||||
const Block & header_,
|
||||
QueryProcessingStage::Enum processed_stage_,
|
||||
QualifiedTableName main_table_,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_)
|
||||
: header(header_),
|
||||
processed_stage{processed_stage_},
|
||||
main_table(std::move(main_table_)),
|
||||
table_func_ptr{nullptr},
|
||||
scalars{scalars_},
|
||||
external_tables{external_tables_}
|
||||
{
|
||||
}
|
||||
@ -46,10 +48,12 @@ SelectStreamFactory::SelectStreamFactory(
|
||||
const Block & header_,
|
||||
QueryProcessingStage::Enum processed_stage_,
|
||||
ASTPtr table_func_ptr_,
|
||||
const Scalars & scalars_,
|
||||
const Tables & external_tables_)
|
||||
: header(header_),
|
||||
processed_stage{processed_stage_},
|
||||
table_func_ptr{table_func_ptr_},
|
||||
scalars{scalars_},
|
||||
external_tables{external_tables_}
|
||||
{
|
||||
}
|
||||
@ -92,7 +96,8 @@ void SelectStreamFactory::createForShard(
|
||||
|
||||
auto emplace_remote_stream = [&]()
|
||||
{
|
||||
auto stream = std::make_shared<RemoteBlockInputStream>(shard_info.pool, query, header, context, nullptr, throttler, external_tables, processed_stage);
|
||||
auto stream = std::make_shared<RemoteBlockInputStream>(
|
||||
shard_info.pool, query, header, context, nullptr, throttler, scalars, external_tables, processed_stage);
|
||||
stream->setPoolMode(PoolMode::GET_MANY);
|
||||
if (!table_func_ptr)
|
||||
stream->setMainTable(main_table);
|
||||
@ -190,8 +195,8 @@ void SelectStreamFactory::createForShard(
|
||||
|
||||
auto lazily_create_stream = [
|
||||
pool = shard_info.pool, shard_num = shard_info.shard_num, query, header = header, query_ast, context, throttler,
|
||||
main_table = main_table, table_func_ptr = table_func_ptr, external_tables = external_tables, stage = processed_stage,
|
||||
local_delay]()
|
||||
main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables,
|
||||
stage = processed_stage, local_delay]()
|
||||
-> BlockInputStreamPtr
|
||||
{
|
||||
auto current_settings = context.getSettingsRef();
|
||||
@ -233,7 +238,7 @@ void SelectStreamFactory::createForShard(
|
||||
connections.emplace_back(std::move(try_result.entry));
|
||||
|
||||
return std::make_shared<RemoteBlockInputStream>(
|
||||
std::move(connections), query, header, context, nullptr, throttler, external_tables, stage);
|
||||
std::move(connections), query, header, context, nullptr, throttler, scalars, external_tables, stage);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -164,7 +164,8 @@ size_t CollectJoinOnKeysMatcher::getTableForIdentifiers(std::vector<const ASTIde
|
||||
|
||||
/// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor.
|
||||
/// In this case it saves membership in IdentifierSemantic.
|
||||
size_t membership = IdentifierSemantic::getMembership(*identifier);
|
||||
auto opt = IdentifierSemantic::getMembership(*identifier);
|
||||
size_t membership = opt ? (*opt + 1) : 0;
|
||||
|
||||
if (!membership)
|
||||
{
|
||||
|
@ -30,6 +30,7 @@
|
||||
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
|
||||
#include <Interpreters/EmbeddedDictionaries.h>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/ExternalModelsLoader.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
@ -88,6 +89,8 @@ namespace ErrorCodes
|
||||
extern const int SESSION_IS_LOCKED;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int SCALAR_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_SCALAR;
|
||||
}
|
||||
|
||||
|
||||
@ -191,7 +194,7 @@ struct ContextShared
|
||||
bool shutdown_called = false;
|
||||
|
||||
/// Do not allow simultaneous execution of DDL requests on the same table.
|
||||
/// database -> table -> (mutex, counter), counter: how many threads are running a query on the table at the same time
|
||||
/// database -> object -> (mutex, counter), counter: how many threads are running a query on the table at the same time
|
||||
/// For the duration of the operation, an element is placed here, and an object is returned,
|
||||
/// which deletes the element in the destructor when counter becomes zero.
|
||||
/// In case the element already exists, waits, when query will be executed in other thread. See class DDLGuard below.
|
||||
@ -792,6 +795,16 @@ bool Context::isTableExist(const String & database_name, const String & table_na
|
||||
&& it->second->isTableExist(*this, table_name);
|
||||
}
|
||||
|
||||
bool Context::isDictionaryExists(const String & database_name, const String & dictionary_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
checkDatabaseAccessRightsImpl(db);
|
||||
|
||||
Databases::const_iterator it = shared->databases.find(db);
|
||||
return shared->databases.end() != it && it->second->isDictionaryExist(*this, dictionary_name);
|
||||
}
|
||||
|
||||
bool Context::isDatabaseExist(const String & database_name) const
|
||||
{
|
||||
@ -807,22 +820,6 @@ bool Context::isExternalTableExist(const String & table_name) const
|
||||
}
|
||||
|
||||
|
||||
void Context::assertTableExists(const String & database_name, const String & table_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
checkDatabaseAccessRightsImpl(db);
|
||||
|
||||
Databases::const_iterator it = shared->databases.find(db);
|
||||
if (shared->databases.end() == it)
|
||||
throw Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
|
||||
|
||||
if (!it->second->isTableExist(*this, table_name))
|
||||
throw Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
}
|
||||
|
||||
|
||||
void Context::assertTableDoesntExist(const String & database_name, const String & table_name, bool check_database_access_rights) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -862,6 +859,21 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const
|
||||
}
|
||||
|
||||
|
||||
const Scalars & Context::getScalars() const
|
||||
{
|
||||
return scalars;
|
||||
}
|
||||
|
||||
|
||||
const Block & Context::getScalar(const String & name) const
|
||||
{
|
||||
auto it = scalars.find(name);
|
||||
if (scalars.end() == it)
|
||||
throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::UNKNOWN_SCALAR);
|
||||
return it->second;
|
||||
}
|
||||
|
||||
|
||||
Tables Context::getExternalTables() const
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -959,6 +971,19 @@ void Context::addExternalTable(const String & table_name, const StoragePtr & sto
|
||||
external_tables[table_name] = std::pair(storage, ast);
|
||||
}
|
||||
|
||||
|
||||
void Context::addScalar(const String & name, const Block & block)
|
||||
{
|
||||
scalars[name] = block;
|
||||
}
|
||||
|
||||
|
||||
bool Context::hasScalar(const String & name) const
|
||||
{
|
||||
return scalars.count(name);
|
||||
}
|
||||
|
||||
|
||||
StoragePtr Context::tryRemoveExternalTable(const String & table_name)
|
||||
{
|
||||
TableAndCreateASTs::const_iterator it = external_tables.find(table_name);
|
||||
@ -1046,9 +1071,10 @@ void Context::addDatabase(const String & database_name, const DatabasePtr & data
|
||||
DatabasePtr Context::detachDatabase(const String & database_name)
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
auto res = getDatabase(database_name);
|
||||
getExternalDictionariesLoader().removeConfigRepository(database_name);
|
||||
shared->databases.erase(database_name);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -1063,6 +1089,17 @@ ASTPtr Context::getCreateTableQuery(const String & database_name, const String &
|
||||
return shared->databases[db]->getCreateTableQuery(*this, table_name);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr Context::getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
String db = resolveDatabase(database_name, current_database);
|
||||
assertDatabaseExists(db);
|
||||
|
||||
return shared->databases[db]->getCreateDictionaryQuery(*this, dictionary_name);
|
||||
}
|
||||
|
||||
ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const
|
||||
{
|
||||
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
|
||||
@ -1308,21 +1345,13 @@ EmbeddedDictionaries & Context::getEmbeddedDictionaries()
|
||||
|
||||
const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(shared->external_dictionaries_mutex);
|
||||
if (shared->external_dictionaries_loader)
|
||||
return *shared->external_dictionaries_loader;
|
||||
}
|
||||
|
||||
const auto & config = getConfigRef();
|
||||
std::lock_guard lock(shared->external_dictionaries_mutex);
|
||||
if (!shared->external_dictionaries_loader)
|
||||
{
|
||||
if (!this->global_context)
|
||||
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(config, "dictionaries_config");
|
||||
shared->external_dictionaries_loader.emplace(std::move(config_repository), *this->global_context);
|
||||
shared->external_dictionaries_loader.emplace(*this->global_context);
|
||||
}
|
||||
return *shared->external_dictionaries_loader;
|
||||
}
|
||||
@ -1341,8 +1370,7 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const
|
||||
if (!this->global_context)
|
||||
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto config_repository = std::make_unique<ExternalLoaderXMLConfigRepository>(getConfigRef(), "models_config");
|
||||
shared->external_models_loader.emplace(std::move(config_repository), *this->global_context);
|
||||
shared->external_models_loader.emplace(*this->global_context);
|
||||
}
|
||||
return *shared->external_models_loader;
|
||||
}
|
||||
@ -2039,7 +2067,7 @@ void Context::dropCompiledExpressionCache() const
|
||||
#endif
|
||||
|
||||
|
||||
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd)
|
||||
void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
shared->bridge_commands.emplace_back(std::move(cmd));
|
||||
|
@ -105,6 +105,9 @@ using InputInitializer = std::function<void(Context &, const StoragePtr &)>;
|
||||
/// Callback for reading blocks of data from client for function input()
|
||||
using InputBlocksReader = std::function<Block(Context &)>;
|
||||
|
||||
/// Scalar results of sub queries
|
||||
using Scalars = std::map<String, Block>;
|
||||
|
||||
/// An empty interface for an arbitrary object that may be attached by a shared pointer
|
||||
/// to query context, when using ClickHouse as a library.
|
||||
struct IHostContext
|
||||
@ -144,6 +147,7 @@ private:
|
||||
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
|
||||
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
|
||||
TableAndCreateASTs external_tables; /// Temporary tables.
|
||||
Scalars scalars;
|
||||
StoragePtr view_source; /// Temporary StorageValues used to generate alias columns for materialized views
|
||||
Tables table_function_results; /// Temporary tables obtained by execution of table functions. Keyed by AST tree id.
|
||||
Context * query_context = nullptr;
|
||||
@ -248,9 +252,9 @@ public:
|
||||
/// Checking the existence of the table/database. Database can be empty - in this case the current database is used.
|
||||
bool isTableExist(const String & database_name, const String & table_name) const;
|
||||
bool isDatabaseExist(const String & database_name) const;
|
||||
bool isDictionaryExists(const String & database_name, const String & dictionary_name) const;
|
||||
bool isExternalTableExist(const String & table_name) const;
|
||||
bool hasDatabaseAccessRights(const String & database_name) const;
|
||||
void assertTableExists(const String & database_name, const String & table_name) const;
|
||||
|
||||
bool hasDictionaryAccessRights(const String & dictionary_name) const;
|
||||
|
||||
@ -264,11 +268,15 @@ public:
|
||||
void assertDatabaseDoesntExist(const String & database_name) const;
|
||||
void checkDatabaseAccessRights(const std::string & database_name) const;
|
||||
|
||||
const Scalars & getScalars() const;
|
||||
const Block & getScalar(const String & name) const;
|
||||
Tables getExternalTables() const;
|
||||
StoragePtr tryGetExternalTable(const String & table_name) const;
|
||||
StoragePtr getTable(const String & database_name, const String & table_name) const;
|
||||
StoragePtr tryGetTable(const String & database_name, const String & table_name) const;
|
||||
void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {});
|
||||
void addScalar(const String & name, const Block & block);
|
||||
bool hasScalar(const String & name) const;
|
||||
StoragePtr tryRemoveExternalTable(const String & table_name);
|
||||
|
||||
StoragePtr executeTableFunction(const ASTPtr & table_expression);
|
||||
@ -355,6 +363,7 @@ public:
|
||||
ASTPtr getCreateTableQuery(const String & database_name, const String & table_name) const;
|
||||
ASTPtr getCreateExternalTableQuery(const String & table_name) const;
|
||||
ASTPtr getCreateDatabaseQuery(const String & database_name) const;
|
||||
ASTPtr getCreateDictionaryQuery(const String & database_name, const String & dictionary_name) const;
|
||||
|
||||
const DatabasePtr getDatabase(const String & database_name) const;
|
||||
DatabasePtr getDatabase(const String & database_name);
|
||||
@ -544,7 +553,7 @@ public:
|
||||
#endif
|
||||
|
||||
/// Add started bridge command. It will be killed after context destruction
|
||||
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd);
|
||||
void addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const;
|
||||
|
||||
IHostContextPtr & getHostContext();
|
||||
const IHostContextPtr & getHostContext() const;
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Interpreters/CrossToInnerJoinVisitor.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/IdentifierSemantic.h>
|
||||
#include <Interpreters/QueryNormalizer.h> // for functionIsInOperator
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
@ -90,11 +90,14 @@ public:
|
||||
using TypeToVisit = const ASTFunction;
|
||||
|
||||
CheckExpressionVisitorData(const std::vector<JoinedTable> & tables_)
|
||||
: tables(tables_)
|
||||
: joined_tables(tables_)
|
||||
, ands_only(true)
|
||||
{}
|
||||
{
|
||||
for (auto & joined : joined_tables)
|
||||
tables.push_back(joined.table);
|
||||
}
|
||||
|
||||
void visit(const ASTFunction & node, ASTPtr & ast)
|
||||
void visit(const ASTFunction & node, const ASTPtr & ast)
|
||||
{
|
||||
if (!ands_only)
|
||||
return;
|
||||
@ -156,7 +159,8 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
const std::vector<JoinedTable> & tables;
|
||||
const std::vector<JoinedTable> & joined_tables;
|
||||
std::vector<DatabaseAndTableWithAlias> tables;
|
||||
std::map<size_t, std::vector<ASTPtr>> asts_to_join_on;
|
||||
bool ands_only;
|
||||
|
||||
@ -180,31 +184,16 @@ private:
|
||||
/// @return table position to attach expression to or 0.
|
||||
size_t checkIdentifiers(const ASTIdentifier & left, const ASTIdentifier & right)
|
||||
{
|
||||
/// {best_match, best_table_pos}
|
||||
std::pair<size_t, size_t> left_best{0, 0};
|
||||
std::pair<size_t, size_t> right_best{0, 0};
|
||||
size_t left_table_pos = 0;
|
||||
bool left_match = IdentifierSemantic::chooseTable(left, tables, left_table_pos);
|
||||
|
||||
for (size_t i = 0; i < tables.size(); ++i)
|
||||
size_t right_table_pos = 0;
|
||||
bool right_match = IdentifierSemantic::chooseTable(right, tables, right_table_pos);
|
||||
|
||||
if (left_match && right_match && (left_table_pos != right_table_pos))
|
||||
{
|
||||
size_t match = IdentifierSemantic::canReferColumnToTable(left, tables[i].table);
|
||||
if (match > left_best.first)
|
||||
{
|
||||
left_best.first = match;
|
||||
left_best.second = i;
|
||||
}
|
||||
|
||||
match = IdentifierSemantic::canReferColumnToTable(right, tables[i].table);
|
||||
if (match > right_best.first)
|
||||
{
|
||||
right_best.first = match;
|
||||
right_best.second = i;
|
||||
}
|
||||
}
|
||||
|
||||
if (left_best.first && right_best.first && (left_best.second != right_best.second))
|
||||
{
|
||||
size_t table_pos = std::max(left_best.second, right_best.second);
|
||||
if (tables[table_pos].canAttachOnExpression())
|
||||
size_t table_pos = std::max(left_table_pos, right_table_pos);
|
||||
if (joined_tables[table_pos].canAttachOnExpression())
|
||||
return table_pos;
|
||||
}
|
||||
return 0;
|
||||
@ -212,27 +201,17 @@ 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;
|
||||
}
|
||||
};
|
||||
|
||||
using CheckExpressionMatcher = OneTypeMatcher<CheckExpressionVisitorData, false>;
|
||||
using CheckExpressionVisitor = InDepthNodeVisitor<CheckExpressionMatcher, true>;
|
||||
using CheckExpressionMatcher = ConstOneTypeMatcher<CheckExpressionVisitorData, false>;
|
||||
using CheckExpressionVisitor = ConstInDepthNodeVisitor<CheckExpressionMatcher, true>;
|
||||
|
||||
|
||||
bool getTables(ASTSelectQuery & select, std::vector<JoinedTable> & joined_tables, size_t & num_comma)
|
||||
@ -314,7 +293,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da
|
||||
return;
|
||||
|
||||
CheckExpressionVisitor::Data visitor_data{joined_tables};
|
||||
CheckExpressionVisitor(visitor_data).visit(select.refWhere());
|
||||
CheckExpressionVisitor(visitor_data).visit(select.where());
|
||||
|
||||
if (visitor_data.complex())
|
||||
return;
|
||||
|
@ -6,14 +6,17 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
#include <Columns/ColumnTuple.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -53,69 +56,98 @@ void ExecuteScalarSubqueriesMatcher::visit(ASTPtr & ast, Data & data)
|
||||
visit(*t, ast, data);
|
||||
}
|
||||
|
||||
/// Converting to literal values might take a fair amount of overhead when the value is large, (e.g.
|
||||
/// Array, BitMap, etc.), This conversion is required for constant folding, index lookup, branch
|
||||
/// elimination. However, these optimizations should never be related to large values, thus we
|
||||
/// blacklist them here.
|
||||
static bool worthConvertingToLiteral(const Block & scalar)
|
||||
{
|
||||
auto scalar_type_name = scalar.safeGetByPosition(0).type->getFamilyName();
|
||||
std::set<String> useless_literal_types = {"Array", "Tuple", "AggregateFunction", "Function", "Set", "LowCardinality"};
|
||||
return !useless_literal_types.count(scalar_type_name);
|
||||
}
|
||||
|
||||
void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr & ast, Data & data)
|
||||
{
|
||||
Context subquery_context = data.context;
|
||||
Settings subquery_settings = data.context.getSettings();
|
||||
subquery_settings.max_result_rows = 1;
|
||||
subquery_settings.extremes = 0;
|
||||
subquery_context.setSettings(subquery_settings);
|
||||
auto hash = subquery.getTreeHash();
|
||||
auto scalar_query_hash_str = toString(hash.first) + "_" + toString(hash.second);
|
||||
|
||||
ASTPtr subquery_select = subquery.children.at(0);
|
||||
BlockIO res = InterpreterSelectWithUnionQuery(
|
||||
subquery_select, subquery_context, SelectQueryOptions(QueryProcessingStage::Complete, data.subquery_depth + 1)).execute();
|
||||
|
||||
Block block;
|
||||
try
|
||||
Block scalar;
|
||||
if (data.context.hasQueryContext() && data.context.getQueryContext().hasScalar(scalar_query_hash_str))
|
||||
scalar = data.context.getQueryContext().getScalar(scalar_query_hash_str);
|
||||
else if (data.scalars.count(scalar_query_hash_str))
|
||||
scalar = data.scalars[scalar_query_hash_str];
|
||||
else
|
||||
{
|
||||
block = res.in->read();
|
||||
Context subquery_context = data.context;
|
||||
Settings subquery_settings = data.context.getSettings();
|
||||
subquery_settings.max_result_rows = 1;
|
||||
subquery_settings.extremes = 0;
|
||||
subquery_context.setSettings(subquery_settings);
|
||||
|
||||
if (!block)
|
||||
ASTPtr subquery_select = subquery.children.at(0);
|
||||
BlockIO res = InterpreterSelectWithUnionQuery(
|
||||
subquery_select, subquery_context, SelectQueryOptions(QueryProcessingStage::Complete, data.subquery_depth + 1)).execute();
|
||||
|
||||
Block block;
|
||||
try
|
||||
{
|
||||
/// Interpret subquery with empty result as Null literal
|
||||
auto ast_new = std::make_unique<ASTLiteral>(Null());
|
||||
ast_new->setAlias(ast->tryGetAlias());
|
||||
ast = std::move(ast_new);
|
||||
return;
|
||||
block = res.in->read();
|
||||
|
||||
if (!block)
|
||||
{
|
||||
/// Interpret subquery with empty result as Null literal
|
||||
auto ast_new = std::make_unique<ASTLiteral>(Null());
|
||||
ast_new->setAlias(ast->tryGetAlias());
|
||||
ast = std::move(ast_new);
|
||||
return;
|
||||
}
|
||||
|
||||
if (block.rows() != 1 || res.in->read())
|
||||
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::TOO_MANY_ROWS)
|
||||
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
|
||||
else
|
||||
throw;
|
||||
}
|
||||
|
||||
if (block.rows() != 1 || res.in->read())
|
||||
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::TOO_MANY_ROWS)
|
||||
throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY);
|
||||
block = materializeBlock(block);
|
||||
size_t columns = block.columns();
|
||||
|
||||
if (columns == 1)
|
||||
scalar = block;
|
||||
else
|
||||
throw;
|
||||
{
|
||||
|
||||
ColumnWithTypeAndName ctn;
|
||||
ctn.type = std::make_shared<DataTypeTuple>(block.getDataTypes());
|
||||
ctn.column = ColumnTuple::create(block.getColumns());
|
||||
scalar.insert(ctn);
|
||||
}
|
||||
}
|
||||
|
||||
size_t columns = block.columns();
|
||||
if (columns == 1)
|
||||
const Settings & settings = data.context.getSettingsRef();
|
||||
|
||||
// Always convert to literals when there is no query context.
|
||||
if (!settings.enable_scalar_subquery_optimization || worthConvertingToLiteral(scalar) || !data.context.hasQueryContext())
|
||||
{
|
||||
auto lit = std::make_unique<ASTLiteral>((*block.safeGetByPosition(0).column)[0]);
|
||||
auto lit = std::make_unique<ASTLiteral>((*scalar.safeGetByPosition(0).column)[0]);
|
||||
lit->alias = subquery.alias;
|
||||
lit->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
|
||||
ast = addTypeConversionToAST(std::move(lit), block.safeGetByPosition(0).type->getName());
|
||||
ast = addTypeConversionToAST(std::move(lit), scalar.safeGetByPosition(0).type->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
auto tuple = std::make_shared<ASTFunction>();
|
||||
tuple->alias = subquery.alias;
|
||||
ast = tuple;
|
||||
tuple->name = "tuple";
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
tuple->arguments = exp_list;
|
||||
tuple->children.push_back(tuple->arguments);
|
||||
|
||||
exp_list->children.resize(columns);
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
exp_list->children[i] = addTypeConversionToAST(
|
||||
std::make_unique<ASTLiteral>((*block.safeGetByPosition(i).column)[0]),
|
||||
block.safeGetByPosition(i).type->getName());
|
||||
}
|
||||
auto func = makeASTFunction("__getScalar", std::make_shared<ASTLiteral>(scalar_query_hash_str));
|
||||
func->alias = subquery.alias;
|
||||
func->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name;
|
||||
ast = std::move(func);
|
||||
}
|
||||
|
||||
data.scalars[scalar_query_hash_str] = std::move(scalar);
|
||||
}
|
||||
|
||||
void ExecuteScalarSubqueriesMatcher::visit(const ASTFunction & func, ASTPtr & ast, Data & data)
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
|
||||
namespace DB
|
||||
@ -36,6 +37,7 @@ public:
|
||||
{
|
||||
const Context & context;
|
||||
size_t subquery_depth;
|
||||
Scalars & scalars;
|
||||
};
|
||||
|
||||
static bool needChildVisit(ASTPtr & node, const ASTPtr &);
|
||||
|
@ -54,7 +54,7 @@
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
#include <Interpreters/misc.h>
|
||||
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
|
||||
|
@ -1,17 +1,16 @@
|
||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Must not acquire Context lock in constructor to avoid possibility of deadlocks.
|
||||
ExternalDictionariesLoader::ExternalDictionariesLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
|
||||
ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_)
|
||||
: ExternalLoader("external dictionary", &Logger::get("ExternalDictionariesLoader"))
|
||||
, context(context_)
|
||||
{
|
||||
addConfigRepository(std::move(config_repository), {"dictionary", "name"});
|
||||
enableAsyncLoading(true);
|
||||
enablePeriodicUpdates(true);
|
||||
}
|
||||
@ -23,4 +22,21 @@ ExternalLoader::LoadablePtr ExternalDictionariesLoader::create(
|
||||
return DictionaryFactory::instance().create(name, config, key_in_config, context);
|
||||
}
|
||||
|
||||
void ExternalDictionariesLoader::addConfigRepository(
|
||||
const std::string & repository_name, std::unique_ptr<IExternalLoaderConfigRepository> config_repository)
|
||||
{
|
||||
ExternalLoader::addConfigRepository(repository_name, std::move(config_repository), {"dictionary", "name"});
|
||||
}
|
||||
|
||||
|
||||
void ExternalDictionariesLoader::addDictionaryWithConfig(
|
||||
const String & dictionary_name, const String & repo_name, const ASTCreateQuery & query, bool load_never_loading) const
|
||||
{
|
||||
ExternalLoader::addObjectAndLoad(
|
||||
dictionary_name, /// names are equal
|
||||
dictionary_name,
|
||||
repo_name,
|
||||
getDictionaryConfigurationFromAST(query),
|
||||
"dictionary", load_never_loading);
|
||||
}
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/IExternalLoaderConfigRepository.h>
|
||||
#include <Interpreters/ExternalLoader.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
@ -19,9 +20,7 @@ public:
|
||||
using DictPtr = std::shared_ptr<const IDictionaryBase>;
|
||||
|
||||
/// Dictionaries will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
|
||||
ExternalDictionariesLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository,
|
||||
Context & context_);
|
||||
ExternalDictionariesLoader(Context & context_);
|
||||
|
||||
DictPtr getDictionary(const std::string & name) const
|
||||
{
|
||||
@ -33,6 +32,18 @@ public:
|
||||
return std::static_pointer_cast<const IDictionaryBase>(tryGetLoadable(name));
|
||||
}
|
||||
|
||||
void addConfigRepository(
|
||||
const std::string & repository_name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository);
|
||||
|
||||
/// Starts reloading of a specified object.
|
||||
void addDictionaryWithConfig(
|
||||
const String & dictionary_name,
|
||||
const String & repo_name,
|
||||
const ASTCreateQuery & query,
|
||||
bool load_never_loading = false) const;
|
||||
|
||||
|
||||
protected:
|
||||
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & key_in_config) const override;
|
||||
|
@ -20,12 +20,29 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Lock mutex only in async mode
|
||||
/// In other case does nothing
|
||||
struct LoadingGuardForAsyncLoad
|
||||
{
|
||||
std::unique_lock<std::mutex> lock;
|
||||
LoadingGuardForAsyncLoad(bool async, std::mutex & mutex)
|
||||
{
|
||||
if (async)
|
||||
lock = std::unique_lock(mutex);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
struct ExternalLoader::ObjectConfig
|
||||
{
|
||||
String config_path;
|
||||
Poco::AutoPtr<Poco::Util::AbstractConfiguration> config;
|
||||
String key_in_config;
|
||||
String repository_name;
|
||||
};
|
||||
|
||||
|
||||
@ -40,26 +57,84 @@ public:
|
||||
}
|
||||
~LoadablesConfigReader() = default;
|
||||
|
||||
void addConfigRepository(std::unique_ptr<IExternalLoaderConfigRepository> repository, const ExternalLoaderConfigSettings & settings)
|
||||
void addConfigRepository(
|
||||
const String & name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> repository,
|
||||
const ExternalLoaderConfigSettings & settings)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
repositories.emplace_back(std::move(repository), std::move(settings));
|
||||
repositories.emplace(name, std::make_pair(std::move(repository), settings));
|
||||
}
|
||||
|
||||
void removeConfigRepository(const String & name)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
repositories.erase(name);
|
||||
}
|
||||
|
||||
using ObjectConfigsPtr = std::shared_ptr<const std::unordered_map<String /* object's name */, ObjectConfig>>;
|
||||
|
||||
/// Reads configuration files.
|
||||
|
||||
/// Reads configurations.
|
||||
ObjectConfigsPtr read()
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
// Check last modification times of files and read those files which are new or changed.
|
||||
if (!readLoadablesInfos())
|
||||
return configs; // Nothing changed, so we can return the previous result.
|
||||
|
||||
return collectConfigs();
|
||||
}
|
||||
|
||||
ObjectConfig updateLoadableInfo(
|
||||
const String & external_name,
|
||||
const String & object_name,
|
||||
const String & repo_name,
|
||||
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
|
||||
const String & key)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
auto it = loadables_infos.find(object_name);
|
||||
if (it == loadables_infos.end())
|
||||
{
|
||||
LoadablesInfos loadable_info;
|
||||
loadables_infos[object_name] = loadable_info;
|
||||
}
|
||||
auto & loadable_info = loadables_infos[object_name];
|
||||
ObjectConfig object_config{object_name, config, key, repo_name};
|
||||
bool found = false;
|
||||
for (auto iter = loadable_info.configs.begin(); iter != loadable_info.configs.end(); ++iter)
|
||||
{
|
||||
if (iter->first == external_name)
|
||||
{
|
||||
iter->second = object_config;
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (!found)
|
||||
loadable_info.configs.emplace_back(external_name, object_config);
|
||||
loadable_info.last_update_time = Poco::Timestamp{}; /// now
|
||||
loadable_info.in_use = true;
|
||||
return object_config;
|
||||
}
|
||||
|
||||
private:
|
||||
struct LoadablesInfos
|
||||
{
|
||||
Poco::Timestamp last_update_time = 0;
|
||||
std::vector<std::pair<String, ObjectConfig>> configs; // Parsed loadable's contents.
|
||||
bool in_use = true; // Whether the `LoadablesInfos` should be destroyed because the correspondent loadable is deleted.
|
||||
};
|
||||
|
||||
/// Collect current configurations
|
||||
ObjectConfigsPtr collectConfigs()
|
||||
{
|
||||
// Generate new result.
|
||||
auto new_configs = std::make_shared<std::unordered_map<String /* object's name */, ObjectConfig>>();
|
||||
for (const auto & [path, loadable_info] : loadables_infos)
|
||||
for (const auto & [path, loadable_info] : loadables_infos)
|
||||
{
|
||||
for (const auto & [name, config] : loadable_info.configs)
|
||||
{
|
||||
@ -81,14 +156,6 @@ public:
|
||||
return configs;
|
||||
}
|
||||
|
||||
private:
|
||||
struct LoadablesInfos
|
||||
{
|
||||
Poco::Timestamp last_update_time = 0;
|
||||
std::vector<std::pair<String, ObjectConfig>> configs; // Parsed file's contents.
|
||||
bool in_use = true; // Whether the ` LoadablesInfos` should be destroyed because the correspondent file is deleted.
|
||||
};
|
||||
|
||||
/// Read files and store them to the map ` loadables_infos`.
|
||||
bool readLoadablesInfos()
|
||||
{
|
||||
@ -100,58 +167,59 @@ private:
|
||||
loadable_info.in_use = false;
|
||||
}
|
||||
|
||||
for (const auto & [repository, settings] : repositories)
|
||||
for (const auto & [repo_name, repo_with_settings] : repositories)
|
||||
{
|
||||
const auto names = repository->getAllLoadablesDefinitionNames();
|
||||
for (const auto & name : names)
|
||||
const auto names = repo_with_settings.first->getAllLoadablesDefinitionNames();
|
||||
for (const auto & loadable_name : names)
|
||||
{
|
||||
auto it = loadables_infos.find(name);
|
||||
if (it != loadables_infos.end())
|
||||
auto it = loadables_infos.find(loadable_name);
|
||||
if (it != loadables_infos.end())
|
||||
{
|
||||
LoadablesInfos & loadable_info = it->second;
|
||||
if (readLoadablesInfo(*repository, name, settings, loadable_info))
|
||||
if (readLoadablesInfo(repo_name, *repo_with_settings.first, loadable_name, repo_with_settings.second, loadable_info))
|
||||
changed = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
LoadablesInfos loadable_info;
|
||||
if (readLoadablesInfo(*repository, name, settings, loadable_info))
|
||||
if (readLoadablesInfo(repo_name, *repo_with_settings.first, loadable_name, repo_with_settings.second, loadable_info))
|
||||
{
|
||||
loadables_infos.emplace(name, std::move(loadable_info));
|
||||
loadables_infos.emplace(loadable_name, std::move(loadable_info));
|
||||
changed = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<String> deleted_files;
|
||||
std::vector<String> deleted_names;
|
||||
for (auto & [path, loadable_info] : loadables_infos)
|
||||
if (!loadable_info.in_use)
|
||||
deleted_files.emplace_back(path);
|
||||
if (!deleted_files.empty())
|
||||
deleted_names.emplace_back(path);
|
||||
if (!deleted_names.empty())
|
||||
{
|
||||
for (const String & deleted_file : deleted_files)
|
||||
loadables_infos.erase(deleted_file);
|
||||
for (const String & deleted_name : deleted_names)
|
||||
loadables_infos.erase(deleted_name);
|
||||
changed = true;
|
||||
}
|
||||
return changed;
|
||||
}
|
||||
|
||||
bool readLoadablesInfo(
|
||||
const String & repo_name,
|
||||
IExternalLoaderConfigRepository & repository,
|
||||
const String & path,
|
||||
const String & object_name,
|
||||
const ExternalLoaderConfigSettings & settings,
|
||||
LoadablesInfos & loadable_info) const
|
||||
{
|
||||
try
|
||||
{
|
||||
if (path.empty() || !repository.exists(path))
|
||||
if (object_name.empty() || !repository.exists(object_name))
|
||||
{
|
||||
LOG_WARNING(log, "config file '" + path + "' does not exist");
|
||||
LOG_WARNING(log, "Config file '" + object_name + "' does not exist");
|
||||
return false;
|
||||
}
|
||||
|
||||
auto update_time_from_repository = repository.getUpdateTime(path);
|
||||
auto update_time_from_repository = repository.getUpdateTime(object_name);
|
||||
|
||||
/// Actually it can't be less, but for sure we check less or equal
|
||||
if (update_time_from_repository <= loadable_info.last_update_time)
|
||||
@ -160,31 +228,31 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
auto file_contents = repository.load(path);
|
||||
auto file_contents = repository.load(object_name);
|
||||
|
||||
/// get all objects' definitions
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
file_contents->keys(keys);
|
||||
|
||||
/// for each object defined in xml config
|
||||
/// for each object defined in repositories
|
||||
std::vector<std::pair<String, ObjectConfig>> configs_from_file;
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
if (!startsWith(key, settings.external_config))
|
||||
{
|
||||
if (!startsWith(key, "comment") && !startsWith(key, "include_from"))
|
||||
LOG_WARNING(log, path << ": file contains unknown node '" << key << "', expected '" << settings.external_config << "'");
|
||||
LOG_WARNING(log, object_name << ": file contains unknown node '" << key << "', expected '" << settings.external_config << "'");
|
||||
continue;
|
||||
}
|
||||
|
||||
String name = file_contents->getString(key + "." + settings.external_name);
|
||||
if (name.empty())
|
||||
String external_name = file_contents->getString(key + "." + settings.external_name);
|
||||
if (external_name.empty())
|
||||
{
|
||||
LOG_WARNING(log, path << ": node '" << key << "' defines " << type_name << " with an empty name. It's not allowed");
|
||||
LOG_WARNING(log, object_name << ": node '" << key << "' defines " << type_name << " with an empty name. It's not allowed");
|
||||
continue;
|
||||
}
|
||||
|
||||
configs_from_file.emplace_back(name, ObjectConfig{path, file_contents, key});
|
||||
configs_from_file.emplace_back(external_name, ObjectConfig{object_name, file_contents, key, repo_name});
|
||||
}
|
||||
|
||||
loadable_info.configs = std::move(configs_from_file);
|
||||
@ -194,16 +262,19 @@ private:
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Failed to read config file '" + path + "'");
|
||||
tryLogCurrentException(log, "Failed to load config for dictionary '" + object_name + "'");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
const String type_name;
|
||||
Logger * log;
|
||||
|
||||
std::mutex mutex;
|
||||
std::vector<std::pair<std::unique_ptr<IExternalLoaderConfigRepository>, ExternalLoaderConfigSettings>> repositories;
|
||||
using RepositoryPtr = std::unique_ptr<IExternalLoaderConfigRepository>;
|
||||
using RepositoryWithSettings = std::pair<RepositoryPtr, ExternalLoaderConfigSettings>;
|
||||
std::unordered_map<String, RepositoryWithSettings> repositories;
|
||||
ObjectConfigsPtr configs;
|
||||
std::unordered_map<String /* config path */, LoadablesInfos> loadables_infos;
|
||||
};
|
||||
@ -267,10 +338,10 @@ public:
|
||||
else
|
||||
{
|
||||
const auto & new_config = new_config_it->second;
|
||||
if (!isSameConfiguration(*info.config.config, info.config.key_in_config, *new_config.config, new_config.key_in_config))
|
||||
if (!isSameConfiguration(*info.object_config.config, info.object_config.key_in_config, *new_config.config, new_config.key_in_config))
|
||||
{
|
||||
/// Configuration has been changed.
|
||||
info.config = new_config;
|
||||
info.object_config = new_config;
|
||||
info.config_changed = true;
|
||||
|
||||
if (info.wasLoading())
|
||||
@ -305,6 +376,12 @@ public:
|
||||
event.notify_all();
|
||||
}
|
||||
|
||||
void setSingleObjectConfigurationWithoutLoading(const String & external_name, const ObjectConfig & config)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
infos.emplace(external_name, Info{config});
|
||||
}
|
||||
|
||||
/// Sets whether all the objects from the configuration should be always loaded (even if they aren't used).
|
||||
void enableAlwaysLoadEverything(bool enable)
|
||||
{
|
||||
@ -326,7 +403,6 @@ public:
|
||||
/// Sets whether the objects should be loaded asynchronously, each loading in a new thread (from the thread pool).
|
||||
void enableAsyncLoading(bool enable)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
enable_async_loading = enable;
|
||||
}
|
||||
|
||||
@ -432,8 +508,17 @@ public:
|
||||
loaded_objects = collectLoadedObjects(filter_by_name);
|
||||
}
|
||||
|
||||
/// Tries to finish loading of the objects for which the specified function returns true.
|
||||
void load(const FilterByNameFunction & filter_by_name, LoadResults & loaded_results, Duration timeout = NO_TIMEOUT)
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
loadImpl(filter_by_name, timeout, lock);
|
||||
loaded_results = collectLoadResults(filter_by_name);
|
||||
}
|
||||
|
||||
/// Tries to finish loading of all the objects during the timeout.
|
||||
void load(Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) { load(allNames, loaded_objects, timeout); }
|
||||
void load(LoadResults & loaded_results, Duration timeout = NO_TIMEOUT) { load(allNames, loaded_results, timeout); }
|
||||
|
||||
/// Starts reloading a specified object.
|
||||
void reload(const String & name, bool load_never_loading = false)
|
||||
@ -441,7 +526,9 @@ public:
|
||||
std::lock_guard lock{mutex};
|
||||
Info * info = getInfo(name);
|
||||
if (!info)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
if (info->wasLoading() || load_never_loading)
|
||||
{
|
||||
@ -539,7 +626,7 @@ public:
|
||||
private:
|
||||
struct Info
|
||||
{
|
||||
Info(const ObjectConfig & config_) : config(config_) {}
|
||||
Info(const ObjectConfig & object_config_) : object_config(object_config_) {}
|
||||
|
||||
bool loaded() const { return object != nullptr; }
|
||||
bool failed() const { return !object && exception; }
|
||||
@ -571,11 +658,12 @@ private:
|
||||
result.exception = exception;
|
||||
result.loading_start_time = loading_start_time;
|
||||
result.loading_duration = loadingDuration();
|
||||
result.origin = config.config_path;
|
||||
result.origin = object_config.config_path;
|
||||
result.repository_name = object_config.repository_name;
|
||||
return result;
|
||||
}
|
||||
|
||||
ObjectConfig config;
|
||||
ObjectConfig object_config;
|
||||
LoadablePtr object;
|
||||
TimePoint loading_start_time;
|
||||
TimePoint loading_end_time;
|
||||
@ -618,8 +706,10 @@ private:
|
||||
LoadResults load_results;
|
||||
load_results.reserve(infos.size());
|
||||
for (const auto & [name, info] : infos)
|
||||
{
|
||||
if (filter_by_name(name))
|
||||
load_results.emplace_back(name, info.loadResult());
|
||||
}
|
||||
return load_results;
|
||||
}
|
||||
|
||||
@ -691,48 +781,15 @@ private:
|
||||
}
|
||||
}
|
||||
|
||||
/// Does the loading, possibly in the separate thread.
|
||||
void doLoading(const String & name, size_t loading_id, bool async)
|
||||
/// Load one object, returns object ptr or exception
|
||||
/// Do not require locking
|
||||
|
||||
std::pair<LoadablePtr, std::exception_ptr> loadOneObject(
|
||||
const String & name,
|
||||
const ObjectConfig & config,
|
||||
bool config_changed,
|
||||
LoadablePtr previous_version)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock;
|
||||
if (async)
|
||||
{
|
||||
setThreadName("ExterLdrJob");
|
||||
lock = std::unique_lock{mutex}; /// If `async == false` the mutex is already locked.
|
||||
}
|
||||
|
||||
SCOPE_EXIT({
|
||||
if (async)
|
||||
{
|
||||
if (!lock.owns_lock())
|
||||
lock.lock();
|
||||
/// Remove the information about the thread after it finishes.
|
||||
auto it = loading_ids.find(loading_id);
|
||||
if (it != loading_ids.end())
|
||||
{
|
||||
it->second.detach();
|
||||
loading_ids.erase(it);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
/// We check here if this is exactly the same loading as we planned to perform.
|
||||
/// This check is necessary because the object could be removed or load with another config before this thread even starts.
|
||||
Info * info = getInfo(name);
|
||||
if (!info || !info->loading() || (info->loading_id != loading_id))
|
||||
return;
|
||||
|
||||
ObjectConfig config = info->config;
|
||||
bool config_changed = info->config_changed;
|
||||
LoadablePtr previous_version = info->object;
|
||||
size_t error_count = info->error_count;
|
||||
|
||||
/// Use `create_function` to perform the actual loading.
|
||||
/// It's much better to do it with `mutex` unlocked because the loading can take a lot of time
|
||||
/// and require access to other objects.
|
||||
if (async)
|
||||
lock.unlock();
|
||||
|
||||
LoadablePtr new_object;
|
||||
std::exception_ptr new_exception;
|
||||
try
|
||||
@ -743,10 +800,45 @@ private:
|
||||
{
|
||||
new_exception = std::current_exception();
|
||||
}
|
||||
return std::make_pair(new_object, new_exception);
|
||||
|
||||
if (!new_object && !new_exception)
|
||||
throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
/// Return single object info, checks loading_id and name
|
||||
std::optional<Info> getSingleObjectInfo(const String & name, size_t loading_id, bool async)
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
Info * info = getInfo(name);
|
||||
if (!info || !info->loading() || (info->loading_id != loading_id))
|
||||
return {};
|
||||
|
||||
return *info;
|
||||
}
|
||||
|
||||
/// Removes object loading_id from loading_ids if it present
|
||||
/// in other case do nothin should by done with lock
|
||||
void finishObjectLoading(size_t loading_id, const LoadingGuardForAsyncLoad &)
|
||||
{
|
||||
auto it = loading_ids.find(loading_id);
|
||||
if (it != loading_ids.end())
|
||||
{
|
||||
it->second.detach();
|
||||
loading_ids.erase(it);
|
||||
}
|
||||
}
|
||||
|
||||
/// Process loading result
|
||||
/// Calculates next update time and process errors
|
||||
void processLoadResult(
|
||||
const String & name,
|
||||
size_t loading_id,
|
||||
LoadablePtr previous_version,
|
||||
LoadablePtr new_object,
|
||||
std::exception_ptr new_exception,
|
||||
size_t error_count,
|
||||
bool async)
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
/// Calculate a new update time.
|
||||
TimePoint next_update_time;
|
||||
try
|
||||
@ -755,7 +847,12 @@ private:
|
||||
++error_count;
|
||||
else
|
||||
error_count = 0;
|
||||
next_update_time = calculateNextUpdateTime(new_object, error_count);
|
||||
|
||||
LoadablePtr object = previous_version;
|
||||
if (new_object)
|
||||
object = new_object;
|
||||
|
||||
next_update_time = calculateNextUpdateTime(object, error_count);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -763,10 +860,8 @@ private:
|
||||
next_update_time = TimePoint::max();
|
||||
}
|
||||
|
||||
/// Lock the mutex again to store the changes.
|
||||
if (async)
|
||||
lock.lock();
|
||||
info = getInfo(name);
|
||||
|
||||
Info * info = getInfo(name);
|
||||
|
||||
/// And again we should check if this is still the same loading as we were doing.
|
||||
/// This is necessary because the object could be removed or load with another config while the `mutex` was unlocked.
|
||||
@ -802,10 +897,38 @@ private:
|
||||
if (new_object)
|
||||
info->config_changed = false;
|
||||
|
||||
/// Notify `event` to recheck conditions in loadImpl() now.
|
||||
if (async)
|
||||
lock.unlock();
|
||||
event.notify_all();
|
||||
finishObjectLoading(loading_id, lock);
|
||||
}
|
||||
|
||||
|
||||
/// Does the loading, possibly in the separate thread.
|
||||
void doLoading(const String & name, size_t loading_id, bool async)
|
||||
{
|
||||
try
|
||||
{
|
||||
/// We check here if this is exactly the same loading as we planned to perform.
|
||||
/// This check is necessary because the object could be removed or load with another config before this thread even starts.
|
||||
std::optional<Info> info = getSingleObjectInfo(name, loading_id, async);
|
||||
if (!info)
|
||||
return;
|
||||
|
||||
/// Use `create_function` to perform the actual loading.
|
||||
/// It's much better to do it with `mutex` unlocked because the loading can take a lot of time
|
||||
/// and require access to other objects.
|
||||
auto [new_object, new_exception] = loadOneObject(name, info->object_config, info->config_changed, info->object);
|
||||
if (!new_object && !new_exception)
|
||||
throw Exception("No object created and no exception raised for " + type_name, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
|
||||
processLoadResult(name, loading_id, info->object, new_object, new_exception, info->error_count, async);
|
||||
event.notify_all();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LoadingGuardForAsyncLoad lock(async, mutex);
|
||||
finishObjectLoading(loading_id, lock);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void cancelLoading(const String & name)
|
||||
@ -844,7 +967,8 @@ private:
|
||||
TimePoint calculateNextUpdateTime(const LoadablePtr & loaded_object, size_t error_count) const
|
||||
{
|
||||
static constexpr auto never = TimePoint::max();
|
||||
if (!error_count)
|
||||
|
||||
if (loaded_object)
|
||||
{
|
||||
if (!loaded_object->supportUpdates())
|
||||
return never;
|
||||
@ -854,8 +978,11 @@ private:
|
||||
if (lifetime.min_sec == 0 || lifetime.max_sec == 0)
|
||||
return never;
|
||||
|
||||
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
return std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
if (!error_count)
|
||||
{
|
||||
std::uniform_int_distribution<UInt64> distribution{lifetime.min_sec, lifetime.max_sec};
|
||||
return std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
|
||||
}
|
||||
}
|
||||
|
||||
return std::chrono::system_clock::now() + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count));
|
||||
@ -870,7 +997,7 @@ private:
|
||||
ObjectConfigsPtr configs;
|
||||
std::unordered_map<String, Info> infos;
|
||||
bool always_load_everything = false;
|
||||
bool enable_async_loading = false;
|
||||
std::atomic<bool> enable_async_loading = false;
|
||||
std::unordered_map<size_t, ThreadFromGlobalPool> loading_ids;
|
||||
size_t next_loading_id = 1; /// should always be > 0
|
||||
mutable pcg64 rnd_engine{randomSeed()};
|
||||
@ -956,12 +1083,19 @@ ExternalLoader::ExternalLoader(const String & type_name_, Logger * log)
|
||||
ExternalLoader::~ExternalLoader() = default;
|
||||
|
||||
void ExternalLoader::addConfigRepository(
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings)
|
||||
const std::string & repository_name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
|
||||
const ExternalLoaderConfigSettings & config_settings)
|
||||
{
|
||||
config_files_reader->addConfigRepository(std::move(config_repository), config_settings);
|
||||
config_files_reader->addConfigRepository(repository_name, std::move(config_repository), config_settings);
|
||||
loading_dispatcher->setConfiguration(config_files_reader->read());
|
||||
}
|
||||
|
||||
void ExternalLoader::removeConfigRepository(const std::string & repository_name)
|
||||
{
|
||||
config_files_reader->removeConfigRepository(repository_name);
|
||||
}
|
||||
|
||||
void ExternalLoader::enableAlwaysLoadEverything(bool enable)
|
||||
{
|
||||
loading_dispatcher->enableAlwaysLoadEverything(enable);
|
||||
@ -1035,23 +1169,52 @@ void ExternalLoader::load(const FilterByNameFunction & filter_by_name, Loadables
|
||||
loading_dispatcher->load(loaded_objects, timeout);
|
||||
}
|
||||
|
||||
|
||||
void ExternalLoader::load(const FilterByNameFunction & filter_by_name, LoadResults & loaded_objects, Duration timeout) const
|
||||
{
|
||||
if (filter_by_name)
|
||||
loading_dispatcher->load(filter_by_name, loaded_objects, timeout);
|
||||
else
|
||||
loading_dispatcher->load(loaded_objects, timeout);
|
||||
}
|
||||
|
||||
|
||||
void ExternalLoader::load(Loadables & loaded_objects, Duration timeout) const
|
||||
{
|
||||
return loading_dispatcher->load(loaded_objects, timeout);
|
||||
}
|
||||
|
||||
void ExternalLoader::reload(const String & name, bool load_never_loading)
|
||||
void ExternalLoader::reload(const String & name, bool load_never_loading) const
|
||||
{
|
||||
loading_dispatcher->setConfiguration(config_files_reader->read());
|
||||
auto configs = config_files_reader->read();
|
||||
loading_dispatcher->setConfiguration(configs);
|
||||
loading_dispatcher->reload(name, load_never_loading);
|
||||
}
|
||||
|
||||
void ExternalLoader::reload(bool load_never_loading)
|
||||
void ExternalLoader::reload(bool load_never_loading) const
|
||||
{
|
||||
loading_dispatcher->setConfiguration(config_files_reader->read());
|
||||
loading_dispatcher->reload(load_never_loading);
|
||||
}
|
||||
|
||||
void ExternalLoader::addObjectAndLoad(
|
||||
const String & name,
|
||||
const String & external_name,
|
||||
const String & repo_name,
|
||||
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
|
||||
const String & key,
|
||||
bool load_never_loading) const
|
||||
{
|
||||
auto object_config = config_files_reader->updateLoadableInfo(external_name, name, repo_name, config, key);
|
||||
loading_dispatcher->setSingleObjectConfigurationWithoutLoading(external_name, object_config);
|
||||
LoadablePtr loaded_object;
|
||||
if (load_never_loading)
|
||||
loading_dispatcher->loadStrict(name, loaded_object);
|
||||
else
|
||||
loading_dispatcher->load(name, loaded_object, Duration::zero());
|
||||
}
|
||||
|
||||
|
||||
ExternalLoader::LoadablePtr ExternalLoader::createObject(
|
||||
const String & name, const ObjectConfig & config, bool config_changed, const LoadablePtr & previous_version) const
|
||||
{
|
||||
|
@ -72,6 +72,7 @@ public:
|
||||
TimePoint loading_start_time;
|
||||
Duration loading_duration;
|
||||
std::exception_ptr exception;
|
||||
std::string repository_name;
|
||||
};
|
||||
|
||||
using LoadResults = std::vector<std::pair<String, LoadResult>>;
|
||||
@ -81,7 +82,12 @@ public:
|
||||
|
||||
/// Adds a repository which will be used to read configurations from.
|
||||
void addConfigRepository(
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository, const ExternalLoaderConfigSettings & config_settings);
|
||||
const std::string & repository_name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository,
|
||||
const ExternalLoaderConfigSettings & config_settings);
|
||||
|
||||
/// Removes a repository which were used to read configurations.
|
||||
void removeConfigRepository(const std::string & repository_name);
|
||||
|
||||
/// Sets whether all the objects from the configuration should be always loaded (even those which are never used).
|
||||
void enableAlwaysLoadEverything(bool enable);
|
||||
@ -132,6 +138,7 @@ public:
|
||||
|
||||
/// Tries to finish loading of the objects for which the specified function returns true.
|
||||
void load(const FilterByNameFunction & filter_by_name, Loadables & loaded_objects, Duration timeout = NO_TIMEOUT) const;
|
||||
void load(const FilterByNameFunction & filter_by_name, LoadResults & load_results, Duration timeout = NO_TIMEOUT) const;
|
||||
Loadables loadAndGet(const FilterByNameFunction & filter_by_name, Duration timeout = NO_TIMEOUT) const { Loadables loaded_objects; load(filter_by_name, loaded_objects, timeout); return loaded_objects; }
|
||||
|
||||
/// Tries to finish loading of all the objects during the timeout.
|
||||
@ -140,16 +147,27 @@ public:
|
||||
/// Starts reloading of a specified object.
|
||||
/// `load_never_loading` specifies what to do if the object has never been loading before.
|
||||
/// The function can either skip it (false) or load for the first time (true).
|
||||
void reload(const String & name, bool load_never_loading = false);
|
||||
/// Also function can load dictionary synchronously
|
||||
void reload(const String & name, bool load_never_loading = false) const;
|
||||
|
||||
|
||||
/// Starts reloading of all the objects.
|
||||
/// `load_never_loading` specifies what to do with the objects which have never been loading before.
|
||||
/// The function can either skip them (false) or load for the first time (true).
|
||||
void reload(bool load_never_loading = false);
|
||||
void reload(bool load_never_loading = false) const;
|
||||
|
||||
protected:
|
||||
virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0;
|
||||
|
||||
/// Reload object with already parsed configuration
|
||||
void addObjectAndLoad(
|
||||
const String & name, /// name of dictionary
|
||||
const String & external_name, /// name of source (example xml-file, may contain more than dictionary)
|
||||
const String & repo_name, /// name of repository (database name, or all xml files)
|
||||
const Poco::AutoPtr<Poco::Util::AbstractConfiguration> & config,
|
||||
const String & key_in_config, /// key where we can start search of loadables (<dictionary>, <model>, etc)
|
||||
bool load_never_loading = false) const;
|
||||
|
||||
private:
|
||||
struct ObjectConfig;
|
||||
|
||||
|
@ -0,0 +1,56 @@
|
||||
#include <Interpreters/ExternalLoaderDatabaseConfigRepository.h>
|
||||
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_DICTIONARY;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
String trimDatabaseName(const std::string & loadable_definition_name, const DatabasePtr database)
|
||||
{
|
||||
const auto & dbname = database->getDatabaseName();
|
||||
if (!startsWith(loadable_definition_name, dbname))
|
||||
throw Exception(
|
||||
"Loadable '" + loadable_definition_name + "' is not from database '" + database->getDatabaseName(), ErrorCodes::UNKNOWN_DICTIONARY);
|
||||
/// dbname.loadable_name
|
||||
///--> remove <---
|
||||
return loadable_definition_name.substr(dbname.length() + 1);
|
||||
}
|
||||
}
|
||||
|
||||
LoadablesConfigurationPtr ExternalLoaderDatabaseConfigRepository::load(const std::string & loadable_definition_name) const
|
||||
{
|
||||
String dictname = trimDatabaseName(loadable_definition_name, database);
|
||||
return getDictionaryConfigurationFromAST(database->getCreateDictionaryQuery(context, dictname)->as<const ASTCreateQuery &>());
|
||||
}
|
||||
|
||||
bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable_definition_name) const
|
||||
{
|
||||
return database->isDictionaryExist(
|
||||
context, trimDatabaseName(loadable_definition_name, database));
|
||||
}
|
||||
|
||||
Poco::Timestamp ExternalLoaderDatabaseConfigRepository::getUpdateTime(const std::string & loadable_definition_name)
|
||||
{
|
||||
return database->getObjectMetadataModificationTime(context, trimDatabaseName(loadable_definition_name, database));
|
||||
}
|
||||
|
||||
std::set<std::string> ExternalLoaderDatabaseConfigRepository::getAllLoadablesDefinitionNames() const
|
||||
{
|
||||
std::set<std::string> result;
|
||||
const auto & dbname = database->getDatabaseName();
|
||||
auto itr = database->getDictionariesIterator(context);
|
||||
while (itr && itr->isValid())
|
||||
{
|
||||
result.insert(dbname + "." + itr->name());
|
||||
itr->next();
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,34 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IExternalLoaderConfigRepository.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Repository from database, which stores dictionary definitions on disk.
|
||||
/// Tracks update time and existance of .sql files through IDatabase.
|
||||
class ExternalLoaderDatabaseConfigRepository : public IExternalLoaderConfigRepository
|
||||
{
|
||||
public:
|
||||
ExternalLoaderDatabaseConfigRepository(const DatabasePtr & database_, const Context & context_)
|
||||
: database(database_)
|
||||
, context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
std::set<std::string> getAllLoadablesDefinitionNames() const override;
|
||||
|
||||
bool exists(const std::string & loadable_definition_name) const override;
|
||||
|
||||
Poco::Timestamp getUpdateTime(const std::string & loadable_definition_name) override;
|
||||
|
||||
LoadablesConfigurationPtr load(const std::string & loadable_definition_name) const override;
|
||||
|
||||
private:
|
||||
DatabasePtr database;
|
||||
Context context;
|
||||
};
|
||||
|
||||
}
|
@ -10,12 +10,10 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
ExternalModelsLoader::ExternalModelsLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository, Context & context_)
|
||||
ExternalModelsLoader::ExternalModelsLoader(Context & context_)
|
||||
: ExternalLoader("external model", &Logger::get("ExternalModelsLoader"))
|
||||
, context(context_)
|
||||
{
|
||||
addConfigRepository(std::move(config_repository), {"model", "name"});
|
||||
enablePeriodicUpdates(true);
|
||||
}
|
||||
|
||||
@ -40,4 +38,8 @@ std::shared_ptr<const IExternalLoadable> ExternalModelsLoader::create(
|
||||
}
|
||||
}
|
||||
|
||||
void ExternalModelsLoader::addConfigRepository(const String & name, std::unique_ptr<IExternalLoaderConfigRepository> config_repository)
|
||||
{
|
||||
ExternalLoader::addConfigRepository(name, std::move(config_repository), {"models", "name"});
|
||||
}
|
||||
}
|
||||
|
@ -18,15 +18,17 @@ public:
|
||||
using ModelPtr = std::shared_ptr<const IModel>;
|
||||
|
||||
/// Models will be loaded immediately and then will be updated in separate thread, each 'reload_period' seconds.
|
||||
ExternalModelsLoader(
|
||||
ExternalLoaderConfigRepositoryPtr config_repository,
|
||||
Context & context_);
|
||||
ExternalModelsLoader(Context & context_);
|
||||
|
||||
ModelPtr getModel(const std::string & name) const
|
||||
{
|
||||
return std::static_pointer_cast<const IModel>(getLoadable(name));
|
||||
}
|
||||
|
||||
void addConfigRepository(const String & name,
|
||||
std::unique_ptr<IExternalLoaderConfigRepository> config_repository);
|
||||
|
||||
|
||||
protected:
|
||||
LoadablePtr create(const std::string & name, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & key_in_config) const override;
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user